From dd10e454ab19b0bf7fd6b4b89e3c22effb8d6977 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 7 Dec 2020 09:59:08 -0500 Subject: [PATCH 01/11] server: improve a miscellaneous comment Release note: None --- pkg/server/init.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/server/init.go b/pkg/server/init.go index ccb08e278862..e1796b8d2616 100644 --- a/pkg/server/init.go +++ b/pkg/server/init.go @@ -541,7 +541,7 @@ func (s *initServer) initializeFirstStoreAfterJoin( type initServerCfg struct { advertiseAddr string binaryMinSupportedVersion roachpb.Version - binaryVersion roachpb.Version // This is what's used for bootstrap. + binaryVersion roachpb.Version // the version used during bootstrap defaultSystemZoneConfig zonepb.ZoneConfig defaultZoneConfig zonepb.ZoneConfig From ca214e96c32edcad73b7f183ed37d851c25672c5 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 7 Dec 2020 10:00:11 -0500 Subject: [PATCH 02/11] server: annotate migration RPC ctx with rpc name Makes for better logging. Release note: None --- pkg/server/migration.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/pkg/server/migration.go b/pkg/server/migration.go index d5f2252dab5b..76765bac3db6 100644 --- a/pkg/server/migration.go +++ b/pkg/server/migration.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" + "github.com/cockroachdb/logtags" "github.com/cockroachdb/redact" ) @@ -39,6 +40,10 @@ var _ serverpb.MigrationServer = &migrationServer{} func (m *migrationServer) ValidateTargetClusterVersion( ctx context.Context, req *serverpb.ValidateTargetClusterVersionRequest, ) (*serverpb.ValidateTargetClusterVersionResponse, error) { + ctx, span := m.server.AnnotateCtxWithSpan(ctx, "validate-cv") + defer span.Finish() + ctx = logtags.AddTag(ctx, "validate-cv", nil) + targetCV := req.ClusterVersion versionSetting := m.server.ClusterSettings().Version @@ -81,6 +86,10 @@ func (m *migrationServer) ValidateTargetClusterVersion( func (m *migrationServer) BumpClusterVersion( ctx context.Context, req *serverpb.BumpClusterVersionRequest, ) (*serverpb.BumpClusterVersionResponse, error) { + ctx, span := m.server.AnnotateCtxWithSpan(ctx, "bump-cv") + defer span.Finish() + ctx = logtags.AddTag(ctx, "bump-cv", nil) + m.Lock() defer m.Unlock() From 6f6e8e56a8b253d576e47448bec9e3398b9d3fb6 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 7 Dec 2020 10:02:21 -0500 Subject: [PATCH 03/11] migration: break down pkg across a few files We can separate out the `Helper`, `Migration`, and various utilities into their own files. We'll add tests for individual components in future commits; the physical separation here sets the foundation for doing so (prototyped in #57445). This commit is purely code movement. Release note: None --- pkg/migration/BUILD.bazel | 1 + pkg/migration/helper.go | 121 +++++++++++++++++++++++++++++ pkg/migration/manager.go | 150 ------------------------------------ pkg/migration/migrations.go | 28 ++++++- pkg/migration/util.go | 29 +++++++ 5 files changed, 178 insertions(+), 151 deletions(-) create mode 100644 pkg/migration/helper.go diff --git a/pkg/migration/BUILD.bazel b/pkg/migration/BUILD.bazel index c02fd0142384..24976336f400 100644 --- a/pkg/migration/BUILD.bazel +++ b/pkg/migration/BUILD.bazel @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "migration", srcs = [ + "helper.go", "manager.go", "migrations.go", "util.go", diff --git a/pkg/migration/helper.go b/pkg/migration/helper.go new file mode 100644 index 000000000000..8b929cab3a1b --- /dev/null +++ b/pkg/migration/helper.go @@ -0,0 +1,121 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package migration + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" +) + +// Helper captures all the primitives required to fully specify a migration. +type Helper struct { + *Manager +} + +// RequiredNodeIDs returns the node IDs for all nodes that are currently part of +// the cluster (i.e. they haven't been decommissioned away). Migrations have the +// pre-requisite that all required nodes are up and running so that we're able +// to execute all relevant node-level operations on them. If any of the nodes +// are found to be unavailable, an error is returned. +func (h *Helper) RequiredNodeIDs(ctx context.Context) ([]roachpb.NodeID, error) { + var nodeIDs []roachpb.NodeID + ls, err := h.nl.GetLivenessesFromKV(ctx) + if err != nil { + return nil, err + } + for _, l := range ls { + if l.Membership.Decommissioned() { + continue + } + live, err := h.nl.IsLive(l.NodeID) + if err != nil { + return nil, err + } + if !live { + return nil, errors.Newf("n%d required, but unavailable", l.NodeID) + } + nodeIDs = append(nodeIDs, l.NodeID) + } + return nodeIDs, nil +} + +// EveryNode invokes the given closure (named by the informational parameter op) +// across every node in the cluster. The mechanism for ensuring that we've done +// so, while accounting for the possibility of new nodes being added to the +// cluster in the interim, is provided by the following structure: +// (a) We'll retrieve the list of node IDs for all nodes in the system +// (b) For each node, we'll invoke the closure +// (c) We'll retrieve the list of node IDs again to account for the +// possibility of a new node being added during (b) +// (d) If there any discrepancies between the list retrieved in (a) +// and (c), we'll invoke the closure each node again +// (e) We'll continue to loop around until the node ID list stabilizes +// +// By the time EveryNode returns, we'll have thus invoked the closure against +// every node in the cluster. +// +// To consider one example of how this primitive is used, let's consider our use +// of it to bump the cluster version. After we return, given all nodes in the +// cluster will have their cluster versions bumped, and future node additions +// will observe the latest version (through the join RPC). This lets us author +// migrations that can assume that a certain version gate has been enabled on +// all nodes in the cluster, and will always be enabled for any new nodes in the +// system. +// +// It may be possible however that right after we return, a new node may join. +// This means that some migrations may have to be split up into two version +// bumps: one that phases out the old version (i.e. stops creation of stale data +// or behavior) and a clean-up version, which removes any vestiges of the stale +// data/behavior, and which, when active, ensures that the old data has vanished +// from the system. This is similar in spirit to how schema changes are split up +// into multiple smaller steps that are carried out sequentially. +func (h *Helper) EveryNode( + ctx context.Context, op string, fn func(context.Context, serverpb.MigrationClient) error, +) error { + nodeIDs, err := h.RequiredNodeIDs(ctx) + if err != nil { + return err + } + + for { + // TODO(irfansharif): We can/should send out these RPCs in parallel. + log.Infof(ctx, "executing op=%s on nodes=%s", op, nodeIDs) + for _, nodeID := range nodeIDs { + conn, err := h.dialer.Dial(ctx, nodeID, rpc.DefaultClass) + if err != nil { + return err + } + client := serverpb.NewMigrationClient(conn) + if err := fn(ctx, client); err != nil { + return err + } + } + + curNodeIDs, err := h.RequiredNodeIDs(ctx) + if err != nil { + return err + } + + if !identical(nodeIDs, curNodeIDs) { + nodeIDs = curNodeIDs + continue + } + + break + } + + return nil +} diff --git a/pkg/migration/manager.go b/pkg/migration/manager.go index 25488e7ec2a1..57f357d9aaaa 100644 --- a/pkg/migration/manager.go +++ b/pkg/migration/manager.go @@ -29,37 +29,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" ) -// Migration defines a program to be executed once every node in the cluster is -// (a) running a specific binary version, and (b) has completed all prior -// migrations. -// -// Each migration is associated with a specific internal cluster version and is -// idempotent in nature. When setting the cluster version (via `SET CLUSTER -// SETTING version`), a manager process determines the set of migrations needed -// to bridge the gap between the current active cluster version, and the target -// one. -// -// To introduce a migration, introduce a version key in pkg/clusterversion, and -// introduce a corresponding internal cluster version for it. See [1] for -// details. Following that, define a Migration in this package and add it to the -// Registry. Be sure to key it in with the new cluster version we just added. -// During cluster upgrades, once the operator is able to set a cluster version -// setting that's past the version that was introduced (typically the major -// release version the migration was introduced in), the manager will execute -// the defined migration before letting the upgrade finalize. -// -// [1]: pkg/clusterversion/cockroach_versions.go -type Migration func(context.Context, *Helper) error - // Manager is the instance responsible for executing migrations across the // cluster. type Manager struct { @@ -69,106 +45,6 @@ type Manager struct { db *kv.DB } -// Helper captures all the primitives required to fully specify a migration. -type Helper struct { - *Manager -} - -// RequiredNodeIDs returns the node IDs for all nodes that are currently part of -// the cluster (i.e. they haven't been decommissioned away). Migrations have the -// pre-requisite that all required nodes are up and running so that we're able -// to execute all relevant node-level operations on them. If any of the nodes -// are found to be unavailable, an error is returned. -func (h *Helper) RequiredNodeIDs(ctx context.Context) ([]roachpb.NodeID, error) { - var nodeIDs []roachpb.NodeID - ls, err := h.nl.GetLivenessesFromKV(ctx) - if err != nil { - return nil, err - } - for _, l := range ls { - if l.Membership.Decommissioned() { - continue - } - live, err := h.nl.IsLive(l.NodeID) - if err != nil { - return nil, err - } - if !live { - return nil, errors.Newf("n%d required, but unavailable", l.NodeID) - } - nodeIDs = append(nodeIDs, l.NodeID) - } - return nodeIDs, nil -} - -// EveryNode invokes the given closure (named by the informational parameter op) -// across every node in the cluster. The mechanism for ensuring that we've done -// so, while accounting for the possibility of new nodes being added to the -// cluster in the interim, is provided by the following structure: -// (a) We'll retrieve the list of node IDs for all nodes in the system -// (b) For each node, we'll invoke the closure -// (c) We'll retrieve the list of node IDs again to account for the -// possibility of a new node being added during (b) -// (d) If there any discrepancies between the list retrieved in (a) -// and (c), we'll invoke the closure each node again -// (e) We'll continue to loop around until the node ID list stabilizes -// -// By the time EveryNode returns, we'll have thus invoked the closure against -// every node in the cluster. -// -// To consider one example of how this primitive is used, let's consider our use -// of it to bump the cluster version. After we return, given all nodes in the -// cluster will have their cluster versions bumped, and future node additions -// will observe the latest version (through the join RPC). This lets us author -// migrations that can assume that a certain version gate has been enabled on -// all nodes in the cluster, and will always be enabled for any new nodes in the -// system. -// -// It may be possible however that right after we return, a new node may join. -// This means that some migrations may have to be split up into two version -// bumps: one that phases out the old version (i.e. stops creation of stale data -// or behavior) and a clean-up version, which removes any vestiges of the stale -// data/behavior, and which, when active, ensures that the old data has vanished -// from the system. This is similar in spirit to how schema changes are split up -// into multiple smaller steps that are carried out sequentially. -func (h *Helper) EveryNode( - ctx context.Context, op string, fn func(context.Context, serverpb.MigrationClient) error, -) error { - nodeIDs, err := h.RequiredNodeIDs(ctx) - if err != nil { - return err - } - - for { - // TODO(irfansharif): We can/should send out these RPCs in parallel. - log.Infof(ctx, "executing op=%s on nodes=%s", op, nodeIDs) - for _, nodeID := range nodeIDs { - conn, err := h.dialer.Dial(ctx, nodeID, rpc.DefaultClass) - if err != nil { - return err - } - client := serverpb.NewMigrationClient(conn) - if err := fn(ctx, client); err != nil { - return err - } - } - - curNodeIDs, err := h.RequiredNodeIDs(ctx) - if err != nil { - return err - } - - if !identical(nodeIDs, curNodeIDs) { - nodeIDs = curNodeIDs - continue - } - - break - } - - return nil -} - // nodeLiveness is the subset of the interface satisfied by CRDB's node liveness // component that the migration manager relies upon. type nodeLiveness interface { @@ -313,29 +189,3 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe return nil } - -// fenceVersionFor constructs the appropriate "fence version" for the given -// cluster version. Fence versions allow the migrations infrastructure to safely -// step through consecutive cluster versions in the presence of nodes (running -// any binary version) being added to the cluster. See the migration manager -// above for intended usage. -// -// Fence versions (and the migrations infrastructure entirely) were introduced -// in the 21.1 release cycle. In the same release cycle, we introduced the -// invariant that new user-defined versions (users being crdb engineers) must -// always have even-numbered Internal versions, thus reserving the odd numbers -// to slot in fence versions for each cluster version. See top-level -// documentation in pkg/clusterversion for more details. -func fenceVersionFor( - ctx context.Context, cv clusterversion.ClusterVersion, -) clusterversion.ClusterVersion { - if (cv.Internal % 2) != 0 { - log.Fatalf(ctx, "only even numbered internal versions allowed, found %s", cv.Version) - } - - // We'll pick the odd internal version preceding the cluster version, - // slotting ourselves right before it. - fenceCV := cv - fenceCV.Internal-- - return fenceCV -} diff --git a/pkg/migration/migrations.go b/pkg/migration/migrations.go index dfae1ee82155..00976d2547ee 100644 --- a/pkg/migration/migrations.go +++ b/pkg/migration/migrations.go @@ -10,7 +10,11 @@ package migration -import "github.com/cockroachdb/cockroach/pkg/clusterversion" +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" +) // Registry defines the global mapping between a cluster version, and the // associated migration. The migration is only executed after a cluster-wide @@ -23,3 +27,25 @@ func init() { // // Registry[clusterversion.ByKey(clusterversion.VersionWhatever)] = WhateverMigration } + +// Migration defines a program to be executed once every node in the cluster is +// (a) running a specific binary version, and (b) has completed all prior +// migrations. +// +// Each migration is associated with a specific internal cluster version and is +// idempotent in nature. When setting the cluster version (via `SET CLUSTER +// SETTING version`), a manager process determines the set of migrations needed +// to bridge the gap between the current active cluster version, and the target +// one. +// +// To introduce a migration, introduce a version key in pkg/clusterversion, and +// introduce a corresponding internal cluster version for it. See [1] for +// details. Following that, define a Migration in this package and add it to the +// Registry. Be sure to key it in with the new cluster version we just added. +// During cluster upgrades, once the operator is able to set a cluster version +// setting that's past the version that was introduced (typically the major +// release version the migration was introduced in), the manager will execute +// the defined migration before letting the upgrade finalize. +// +// [1]: pkg/clusterversion/cockroach_versions.go +type Migration func(context.Context, *Helper) error diff --git a/pkg/migration/util.go b/pkg/migration/util.go index e638dff8d5f4..b9c64405df20 100644 --- a/pkg/migration/util.go +++ b/pkg/migration/util.go @@ -11,9 +11,12 @@ package migration import ( + "context" "sort" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/log" ) // identical returns whether or not two lists of node IDs are identical as sets. @@ -35,3 +38,29 @@ func identical(a, b []roachpb.NodeID) bool { } return true } + +// fenceVersionFor constructs the appropriate "fence version" for the given +// cluster version. Fence versions allow the migrations infrastructure to safely +// step through consecutive cluster versions in the presence of nodes (running +// any binary version) being added to the cluster. See the migration manager +// above for intended usage. +// +// Fence versions (and the migrations infrastructure entirely) were introduced +// in the 21.1 release cycle. In the same release cycle, we introduced the +// invariant that new user-defined versions (users being crdb engineers) must +// always have even-numbered Internal versions, thus reserving the odd numbers +// to slot in fence versions for each cluster version. See top-level +// documentation in pkg/clusterversion for more details. +func fenceVersionFor( + ctx context.Context, cv clusterversion.ClusterVersion, +) clusterversion.ClusterVersion { + if (cv.Internal % 2) != 0 { + log.Fatalf(ctx, "only even numbered internal versions allowed, found %s", cv.Version) + } + + // We'll pick the odd internal version preceding the cluster version, + // slotting ourselves right before it. + fenceCV := cv + fenceCV.Internal-- + return fenceCV +} From c328fb07f13b77603cb2e403a76ddd99e396c33f Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 7 Dec 2020 11:15:23 -0500 Subject: [PATCH 04/11] migration: improve documentation for EveryNode It's clearer to talk explicitly in terms of causality. Release note: None --- pkg/migration/helper.go | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/pkg/migration/helper.go b/pkg/migration/helper.go index 8b929cab3a1b..5632d40e9b23 100644 --- a/pkg/migration/helper.go +++ b/pkg/migration/helper.go @@ -53,8 +53,8 @@ func (h *Helper) RequiredNodeIDs(ctx context.Context) ([]roachpb.NodeID, error) } // EveryNode invokes the given closure (named by the informational parameter op) -// across every node in the cluster. The mechanism for ensuring that we've done -// so, while accounting for the possibility of new nodes being added to the +// across every node in the cluster[*]. The mechanism for ensuring that we've +// done so, while accounting for the possibility of new nodes being added to the // cluster in the interim, is provided by the following structure: // (a) We'll retrieve the list of node IDs for all nodes in the system // (b) For each node, we'll invoke the closure @@ -64,8 +64,13 @@ func (h *Helper) RequiredNodeIDs(ctx context.Context) ([]roachpb.NodeID, error) // and (c), we'll invoke the closure each node again // (e) We'll continue to loop around until the node ID list stabilizes // -// By the time EveryNode returns, we'll have thus invoked the closure against -// every node in the cluster. +// [*]: We can be a bit more precise here. What EveryNode gives us is a strict +// causal happened-before relation between running the given closure against +// every node that's currently a member of the cluster, and the next node that +// joins the cluster. Put another way: using EveryNode callers will have managed +// to run something against all nodes without a new node joining half-way +// through (which could have allowed it to pick up some state off one of the +// existing nodes that hadn't heard from us yet). // // To consider one example of how this primitive is used, let's consider our use // of it to bump the cluster version. After we return, given all nodes in the @@ -75,8 +80,8 @@ func (h *Helper) RequiredNodeIDs(ctx context.Context) ([]roachpb.NodeID, error) // all nodes in the cluster, and will always be enabled for any new nodes in the // system. // -// It may be possible however that right after we return, a new node may join. -// This means that some migrations may have to be split up into two version +// Given that it'll always be possible for new nodes to join after an EveryNode +// round, it means that some migrations may have to be split up into two version // bumps: one that phases out the old version (i.e. stops creation of stale data // or behavior) and a clean-up version, which removes any vestiges of the stale // data/behavior, and which, when active, ensures that the old data has vanished From 3412dd3db5c102145cddcadca2841795adaca4bd Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 7 Dec 2020 11:20:40 -0500 Subject: [PATCH 05/11] migration: introduce a more structured type for `Migration`s We re-define what the Migration type is to be able to annotate it a description. We'll later use this description when populating the `system.migrations` table (originally prototyped in #57445). Release note: None --- pkg/migration/migrations.go | 58 +++++++++++++++++++++++++++---------- 1 file changed, 43 insertions(+), 15 deletions(-) diff --git a/pkg/migration/migrations.go b/pkg/migration/migrations.go index 00976d2547ee..d664e93cb61a 100644 --- a/pkg/migration/migrations.go +++ b/pkg/migration/migrations.go @@ -34,18 +34,46 @@ func init() { // // Each migration is associated with a specific internal cluster version and is // idempotent in nature. When setting the cluster version (via `SET CLUSTER -// SETTING version`), a manager process determines the set of migrations needed -// to bridge the gap between the current active cluster version, and the target -// one. -// -// To introduce a migration, introduce a version key in pkg/clusterversion, and -// introduce a corresponding internal cluster version for it. See [1] for -// details. Following that, define a Migration in this package and add it to the -// Registry. Be sure to key it in with the new cluster version we just added. -// During cluster upgrades, once the operator is able to set a cluster version -// setting that's past the version that was introduced (typically the major -// release version the migration was introduced in), the manager will execute -// the defined migration before letting the upgrade finalize. -// -// [1]: pkg/clusterversion/cockroach_versions.go -type Migration func(context.Context, *Helper) error +// SETTING version`), the manager process determines the set of migrations +// needed to bridge the gap between the current active cluster version, and the +// target one. See [1] for where that happens. +// +// To introduce a migration, start by adding version key to pkg/clusterversion +// and introducing a corresponding internal cluster version for it. See [2] for +// more details. Following that, define a Migration in this package and add it +// to the registry. Be sure to key it in with the new cluster version we just +// added. During cluster upgrades, once the operator is able to set a cluster +// version setting that's past the version that was introduced (typically the +// major release version the migration was introduced in), the manager will +// execute the defined migration before letting the upgrade finalize. +// +// If the migration requires below-Raft level changes ([3] is one example), +// you'll need to add a version switch and the relevant KV-level migration in +// [4]. See IterateRangeDescriptors and the Migrate KV request for more details. +// +// [1]: `(*Manager).Migrate` +// [2]: pkg/clusterversion/cockroach_versions.go +// [3]: TruncatedStateMigration +// [4]: pkg/kv/kvserver/batch_eval/cmd_migrate.go +// +// TODO(irfansharif): [3] and [4] are currently referring to what was prototyped +// in #57445. Once that makes its way into master, this TODO can be removed. +type Migration struct { + cv clusterversion.ClusterVersion + fn migrationFn + desc string +} + +type migrationFn func(context.Context, *Helper) error + +// Run kickstarts the actual migration process. It's responsible for recording +// the ongoing status of the migration into a system table. +// +// TODO(irfansharif): Introduce a `system.migrations` table, and populate it here. +func (m *Migration) Run(ctx context.Context, h *Helper) (err error) { + if err := m.fn(ctx, h); err != nil { + return err + } + + return nil +} From 01ded2d4d9cbf3106efb42a9ea7727364a8a47e9 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 7 Dec 2020 11:23:18 -0500 Subject: [PATCH 06/11] migration: re-define the migration registration process We make it a bit more ergonomic (this revision was originally prototyped in #57445). Release note: None --- pkg/migration/manager.go | 13 ++++++++++--- pkg/migration/migrations.go | 9 ++++----- pkg/migration/util.go | 10 ++++++++++ 3 files changed, 24 insertions(+), 8 deletions(-) diff --git a/pkg/migration/manager.go b/pkg/migration/manager.go index 57f357d9aaaa..cc129feb3c00 100644 --- a/pkg/migration/manager.go +++ b/pkg/migration/manager.go @@ -179,12 +179,19 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe } } - // TODO(irfansharif): We'll want to retrieve the right migration off of - // our registry of migrations, if any, and execute it. // TODO(irfansharif): We'll want to be able to override which migration // is retrieved here within tests. We could make the registry be a part // of the manager, and all tests to provide their own. - _ = Registry[clusterVersion] + + // Finally, run the actual migration. + migration, ok := registry[clusterVersion] + if !ok { + log.Infof(ctx, "no migration registered for %s, skipping", clusterVersion) + continue + } + if err := migration.Run(ctx, h); err != nil { + return err + } } return nil diff --git a/pkg/migration/migrations.go b/pkg/migration/migrations.go index d664e93cb61a..9382ef0abe79 100644 --- a/pkg/migration/migrations.go +++ b/pkg/migration/migrations.go @@ -16,16 +16,15 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" ) -// Registry defines the global mapping between a cluster version, and the +// registry defines the global mapping between a cluster version and the // associated migration. The migration is only executed after a cluster-wide -// bump of the version gate. -var Registry = make(map[clusterversion.ClusterVersion]Migration) +// bump of the corresponding version gate. +var registry = make(map[clusterversion.ClusterVersion]Migration) func init() { // TODO(irfansharif): We'll want to register individual migrations with // specific internal cluster versions here. - // - // Registry[clusterversion.ByKey(clusterversion.VersionWhatever)] = WhateverMigration + _ = register // register(clusterversion.WhateverMigration, WhateverMigration, "whatever migration") } // Migration defines a program to be executed once every node in the cluster is diff --git a/pkg/migration/util.go b/pkg/migration/util.go index b9c64405df20..b7ab5ed6c13e 100644 --- a/pkg/migration/util.go +++ b/pkg/migration/util.go @@ -64,3 +64,13 @@ func fenceVersionFor( fenceCV.Internal-- return fenceCV } + +// register is a short hand to register a given migration within the global +// registry. +func register(key clusterversion.Key, fn migrationFn, desc string) { + cv := clusterversion.ClusterVersion{Version: clusterversion.ByKey(key)} + if _, ok := registry[cv]; ok { + log.Fatalf(context.Background(), "doubly registering migration for %s", cv) + } + registry[cv] = Migration{cv: cv, fn: fn, desc: desc} +} From 0ddde2e8db3056fa3f39c248920f1849b52c4b30 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 7 Dec 2020 11:28:34 -0500 Subject: [PATCH 07/11] migration: shimmy in a `cluster` interface for testability To facilitate testing `Helper` in isolation, we introduce a `cluster` interface that we'll mock out in tests. It's through this interface that the migration infrastructure will able to dial out to a specific node, grab hold of a kv.DB instance, and retrieve the current cluster membership. Part of diff also downgrades `RequiredNodes` from being a first class primitive, instead tucking it away for internal usage only. Given retrieving the cluster membership made no guarantees about new nodes being added to the cluster, it's entirely possible for that to happen concurrently with it. Appropriate usage then entailed wrapping it under a stabilizing loop, like we do so in `EveryNode`. This tells us there's no need to expose it directly to migration authors. Release note: None --- pkg/migration/BUILD.bazel | 27 ++- pkg/migration/helper.go | 132 +++++++++++---- pkg/migration/helper_test.go | 290 ++++++++++++++++++++++++++++++++ pkg/migration/manager.go | 3 +- pkg/migration/util.go | 63 ++++++- pkg/migration/util_test.go | 87 ++++++++++ pkg/testutils/lint/lint_test.go | 1 + 7 files changed, 559 insertions(+), 44 deletions(-) create mode 100644 pkg/migration/helper_test.go create mode 100644 pkg/migration/util_test.go diff --git a/pkg/migration/BUILD.bazel b/pkg/migration/BUILD.bazel index 24976336f400..633153491c15 100644 --- a/pkg/migration/BUILD.bazel +++ b/pkg/migration/BUILD.bazel @@ -1,4 +1,4 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "migration", @@ -19,8 +19,33 @@ go_library( "//pkg/rpc/nodedialer", "//pkg/server/serverpb", "//pkg/sql", + "//pkg/sql/sqlutil", "//pkg/util/log", "//vendor/github.com/cockroachdb/errors", "//vendor/github.com/cockroachdb/logtags", + "//vendor/github.com/cockroachdb/redact", + "//vendor/google.golang.org/grpc", + ], +) + +go_test( + name = "migration_test", + srcs = [ + "helper_test.go", + "util_test.go", + ], + embed = [":migration"], + deps = [ + "//pkg/clusterversion", + "//pkg/kv", + "//pkg/kv/kvserver/liveness/livenesspb", + "//pkg/roachpb", + "//pkg/server/serverpb", + "//pkg/sql/sqlutil", + "//pkg/testutils", + "//pkg/util/leaktest", + "//pkg/util/syncutil", + "//vendor/github.com/cockroachdb/errors", + "//vendor/google.golang.org/grpc", ], ) diff --git a/pkg/migration/helper.go b/pkg/migration/helper.go index 5632d40e9b23..4a539073bb38 100644 --- a/pkg/migration/helper.go +++ b/pkg/migration/helper.go @@ -13,43 +13,56 @@ package migration import ( "context" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" + "google.golang.org/grpc" ) // Helper captures all the primitives required to fully specify a migration. type Helper struct { - *Manager + c cluster + cv clusterversion.ClusterVersion } -// RequiredNodeIDs returns the node IDs for all nodes that are currently part of -// the cluster (i.e. they haven't been decommissioned away). Migrations have the -// pre-requisite that all required nodes are up and running so that we're able -// to execute all relevant node-level operations on them. If any of the nodes -// are found to be unavailable, an error is returned. -func (h *Helper) RequiredNodeIDs(ctx context.Context) ([]roachpb.NodeID, error) { - var nodeIDs []roachpb.NodeID - ls, err := h.nl.GetLivenessesFromKV(ctx) - if err != nil { - return nil, err - } - for _, l := range ls { - if l.Membership.Decommissioned() { - continue - } - live, err := h.nl.IsLive(l.NodeID) - if err != nil { - return nil, err - } - if !live { - return nil, errors.Newf("n%d required, but unavailable", l.NodeID) - } - nodeIDs = append(nodeIDs, l.NodeID) - } - return nodeIDs, nil +// cluster mediates access to the crdb cluster. +type cluster interface { + // nodes returns the IDs and epochs for all nodes that are currently part of + // the cluster (i.e. they haven't been decommissioned away). Migrations have + // the pre-requisite that all nodes are up and running so that we're able to + // execute all relevant node-level operations on them. If any of the nodes + // are found to be unavailable, an error is returned. + // + // It's important to note that this makes no guarantees about new nodes + // being added to the cluster. It's entirely possible for that to happen + // concurrently with the retrieval of the current set of nodes. Appropriate + // usage of this entails wrapping it under a stabilizing loop, like we do in + // EveryNode. + nodes(ctx context.Context) (nodes, error) + + // dial returns a grpc connection to the given node. + dial(context.Context, roachpb.NodeID) (*grpc.ClientConn, error) + + // db provides access the kv.DB instance backing the cluster. + // + // TODO(irfansharif): We could hide the kv.DB instance behind an interface + // to expose only relevant, vetted bits of kv.DB. It'll make our tests less + // "integration-ey". + db() *kv.DB + + // executor provides access to an internal executor instance to run + // arbitrary SQL statements. + executor() sqlutil.InternalExecutor +} + +func newHelper(c cluster, cv clusterversion.ClusterVersion) *Helper { + return &Helper{c: c, cv: cv} } // EveryNode invokes the given closure (named by the informational parameter op) @@ -90,16 +103,16 @@ func (h *Helper) RequiredNodeIDs(ctx context.Context) ([]roachpb.NodeID, error) func (h *Helper) EveryNode( ctx context.Context, op string, fn func(context.Context, serverpb.MigrationClient) error, ) error { - nodeIDs, err := h.RequiredNodeIDs(ctx) + ns, err := h.c.nodes(ctx) if err != nil { return err } for { // TODO(irfansharif): We can/should send out these RPCs in parallel. - log.Infof(ctx, "executing op=%s on nodes=%s", op, nodeIDs) - for _, nodeID := range nodeIDs { - conn, err := h.dialer.Dial(ctx, nodeID, rpc.DefaultClass) + log.Infof(ctx, "executing op=%s on nodes=%s", op, ns) + for _, node := range ns { + conn, err := h.c.dial(ctx, node.id) if err != nil { return err } @@ -109,13 +122,14 @@ func (h *Helper) EveryNode( } } - curNodeIDs, err := h.RequiredNodeIDs(ctx) + curNodes, err := h.c.nodes(ctx) if err != nil { return err } - if !identical(nodeIDs, curNodeIDs) { - nodeIDs = curNodeIDs + if ok, diff := ns.identical(curNodes); !ok { + log.Infof(ctx, "%s, retrying", diff) + ns = curNodes continue } @@ -124,3 +138,55 @@ func (h *Helper) EveryNode( return nil } + +// DB provides exposes the underlying *kv.DB instance. +func (h *Helper) DB() *kv.DB { + return h.c.db() +} + +type clusterImpl struct { + nl nodeLiveness + exec sqlutil.InternalExecutor + dialer *nodedialer.Dialer + kvDB *kv.DB +} + +var _ cluster = &clusterImpl{} + +// nodes implements the cluster interface. +func (c *clusterImpl) nodes(ctx context.Context) (nodes, error) { + var ns []node + ls, err := c.nl.GetLivenessesFromKV(ctx) + if err != nil { + return nil, err + } + for _, l := range ls { + if l.Membership.Decommissioned() { + continue + } + live, err := c.nl.IsLive(l.NodeID) + if err != nil { + return nil, err + } + if !live { + return nil, errors.Newf("n%d required, but unavailable", l.NodeID) + } + ns = append(ns, node{id: l.NodeID, epoch: l.Epoch}) + } + return ns, nil +} + +// dial implements the cluster interface. +func (c *clusterImpl) dial(ctx context.Context, id roachpb.NodeID) (*grpc.ClientConn, error) { + return c.dialer.Dial(ctx, id, rpc.DefaultClass) +} + +// db implements the cluster interface. +func (c *clusterImpl) db() *kv.DB { + return c.kvDB +} + +// executor implements the cluster interface. +func (c *clusterImpl) executor() sqlutil.InternalExecutor { + return c.exec +} diff --git a/pkg/migration/helper_test.go b/pkg/migration/helper_test.go new file mode 100644 index 000000000000..457208bc9ee4 --- /dev/null +++ b/pkg/migration/helper_test.go @@ -0,0 +1,290 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package migration + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/errors" + "google.golang.org/grpc" +) + +func TestHelperEveryNode(t *testing.T) { + defer leaktest.AfterTest(t) + + cv := clusterversion.ClusterVersion{} + ctx := context.Background() + var mu syncutil.Mutex + const numNodes = 3 + + t.Run("with-node-addition", func(t *testing.T) { + // Add a node mid-way through execution. We expect EveryNode to start + // over from scratch and include the newly added node. + tc := TestingNewCluster(numNodes, nil, nil) + h := newHelper(tc, cv) + opCount := 0 + err := h.EveryNode(ctx, "dummy-op", func(context.Context, serverpb.MigrationClient) error { + mu.Lock() + defer mu.Unlock() + + opCount++ + if opCount == numNodes { + tc.addNode() + } + + return nil + }) + if err != nil { + t.Fatal(err) + } + + if exp := numNodes*2 + 1; exp != opCount { + t.Fatalf("expected closure to be invoked %d times, got %d", exp, opCount) + } + }) + + t.Run("with-node-restart", func(t *testing.T) { + // Restart a node mid-way through execution. We expect EveryNode to + // start over from scratch and include the restarted node. + tc := TestingNewCluster(numNodes, nil, nil) + h := newHelper(tc, cv) + opCount := 0 + err := h.EveryNode(ctx, "dummy-op", func(context.Context, serverpb.MigrationClient) error { + mu.Lock() + defer mu.Unlock() + + opCount++ + if opCount == numNodes { + tc.restartNode(2) + } + + return nil + }) + if err != nil { + t.Fatal(err) + } + + if exp := numNodes * 2; exp != opCount { + t.Fatalf("expected closure to be invoked %d times, got %d", exp, opCount) + } + }) + + t.Run("with-node-down", func(t *testing.T) { + // Down a node mid-way through execution. We expect EveryNode to error + // out. + const downedNode = 2 + tc := TestingNewCluster(numNodes, nil, nil) + tc.downNode(downedNode) + expRe := fmt.Sprintf("n%d required, but unavailable", downedNode) + h := newHelper(tc, cv) + if err := h.EveryNode(ctx, "dummy-op", func(context.Context, serverpb.MigrationClient) error { + return nil + }); !testutils.IsError(err, expRe) { + t.Fatalf("expected error %q, got %q", expRe, err) + } + + tc.healCluster() + if err := h.EveryNode(ctx, "dummy-op", func(context.Context, serverpb.MigrationClient) error { + return nil + }); err != nil { + t.Fatal(err) + } + }) +} + +func TestClusterNodes(t *testing.T) { + defer leaktest.AfterTest(t) + + ctx := context.Background() + const numNodes = 3 + + t.Run("retrieve-all", func(t *testing.T) { + nl := newTestNodeLiveness(numNodes) + c := clusterImpl{nl: nl} + + ns, err := c.nodes(ctx) + if err != nil { + t.Fatal(err) + } + + if got := len(ns); got != numNodes { + t.Fatalf("expected %d nodes, got %d", numNodes, got) + } + + for i := range ns { + if exp := roachpb.NodeID(i + 1); exp != ns[i].id { + t.Fatalf("expected to find node ID %s, got %s", exp, ns[i].id) + } + if ns[i].epoch != 1 { + t.Fatalf("expected to find epoch=1, got %d", ns[i].epoch) + } + } + }) + + t.Run("ignore-decommissioned", func(t *testing.T) { + nl := newTestNodeLiveness(numNodes) + c := clusterImpl{nl: nl} + const decommissionedNode = 3 + nl.decommission(decommissionedNode) + + ns, err := c.nodes(ctx) + if err != nil { + t.Fatal(err) + } + + if got := len(ns); got != numNodes-1 { + t.Fatalf("expected %d nodes, got %d", numNodes-1, got) + } + + for i := range ns { + if exp := roachpb.NodeID(i + 1); exp != ns[i].id { + t.Fatalf("expected to find node ID %s, got %s", exp, ns[i].id) + } + if ns[i].epoch != 1 { + t.Fatalf("expected to find epoch=1, got %d", ns[i].epoch) + } + } + }) + + t.Run("error-if-down", func(t *testing.T) { + nl := newTestNodeLiveness(numNodes) + c := clusterImpl{nl: nl} + const downedNode = 3 + nl.down(downedNode) + + _, err := c.nodes(ctx) + expRe := fmt.Sprintf("n%d required, but unavailable", downedNode) + if !testutils.IsError(err, expRe) { + t.Fatalf("expected error %q, got %q", expRe, err) + } + }) +} + +type testClusterImpl struct { + ns nodes + err error + kvDB *kv.DB + exec sqlutil.InternalExecutor +} + +var _ cluster = &testClusterImpl{} + +func TestingNewCluster(numNodes int, db *kv.DB, exec sqlutil.InternalExecutor) *testClusterImpl { + tc := &testClusterImpl{ + kvDB: db, + exec: exec, + } + for i := 0; i < numNodes; i++ { + tc.addNode() + } + + return tc +} + +func (t *testClusterImpl) nodes(context.Context) (nodes, error) { + ns := make([]node, len(t.ns)) + for i, n := range t.ns { + ns[i] = n + } + return ns, t.err +} + +func (t *testClusterImpl) dial(context.Context, roachpb.NodeID) (*grpc.ClientConn, error) { + return nil, nil +} + +func (t *testClusterImpl) db() *kv.DB { + return t.kvDB +} + +func (t *testClusterImpl) executor() sqlutil.InternalExecutor { + return t.exec +} + +func (t *testClusterImpl) addNode() { + t.ns = append(t.ns, node{id: roachpb.NodeID(len(t.ns) + 1), epoch: 1}) +} + +func (t *testClusterImpl) restartNode(id int) { + for i := range t.ns { + if t.ns[i].id == roachpb.NodeID(id) { + t.ns[i].epoch++ + break + } + } +} + +func (t *testClusterImpl) downNode(id int) { + t.err = errors.Newf("n%d required, but unavailable", id) +} + +func (t *testClusterImpl) healCluster() { + t.err = nil +} + +type testNodeLivenessImpl struct { + ls []livenesspb.Liveness + dead map[roachpb.NodeID]struct{} +} + +var _ nodeLiveness = &testNodeLivenessImpl{} + +func newTestNodeLiveness(numNodes int) *testNodeLivenessImpl { + nl := &testNodeLivenessImpl{ + ls: make([]livenesspb.Liveness, numNodes), + dead: make(map[roachpb.NodeID]struct{}), + } + for i := 0; i < numNodes; i++ { + nl.ls[i] = livenesspb.Liveness{ + NodeID: roachpb.NodeID(i + 1), + Epoch: 1, + Membership: livenesspb.MembershipStatus_ACTIVE, + } + } + return nl +} + +func (t *testNodeLivenessImpl) GetLivenessesFromKV(context.Context) ([]livenesspb.Liveness, error) { + return t.ls, nil +} + +func (t *testNodeLivenessImpl) IsLive(id roachpb.NodeID) (bool, error) { + _, dead := t.dead[id] + return !dead, nil +} + +func (t *testNodeLivenessImpl) decommission(id roachpb.NodeID) { + for i := range t.ls { + if t.ls[i].NodeID == id { + t.ls[i].Membership = livenesspb.MembershipStatus_DECOMMISSIONED + break + } + } +} + +func (t *testNodeLivenessImpl) down(id roachpb.NodeID) { + t.dead[id] = struct{}{} +} + +func TestingNewHelper(c cluster, cv clusterversion.ClusterVersion) *Helper { + return &Helper{c: c, cv: cv} +} diff --git a/pkg/migration/manager.go b/pkg/migration/manager.go index cc129feb3c00..e78f2e61fb92 100644 --- a/pkg/migration/manager.go +++ b/pkg/migration/manager.go @@ -99,7 +99,8 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe log.Infof(ctx, "migrating cluster from %s to %s (stepping through %s)", from, to, clusterVersions) for _, clusterVersion := range clusterVersions { - h := &Helper{Manager: m} + cluster := &clusterImpl{nl: m.nl, dialer: m.dialer, exec: m.executor, kvDB: m.db} + h := newHelper(cluster, clusterVersion) // Push out the version gate to every node in the cluster. Each node // will persist the version, bump the local version gates, and then diff --git a/pkg/migration/util.go b/pkg/migration/util.go index b7ab5ed6c13e..ab7f24fd9f96 100644 --- a/pkg/migration/util.go +++ b/pkg/migration/util.go @@ -12,31 +12,76 @@ package migration import ( "context" + "fmt" "sort" + "strings" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/redact" ) -// identical returns whether or not two lists of node IDs are identical as sets. -func identical(a, b []roachpb.NodeID) bool { +// node captures the relevant bits of each node as it pertains to the migration +// infrastructure. +type node struct { + id roachpb.NodeID + epoch int64 +} + +// nodes is a collection of node objects. +type nodes []node + +// identical returns whether or not two lists of nodes are identical as sets, +// and if not, what changed. The textual diff is only to be used for logging +// purposes. +func (ns nodes) identical(other nodes) (ok bool, diff string) { + a, b := ns, other if len(a) != len(b) { - return false + if len(a) < len(b) { + diff = "node added to the cluster" + } else { + diff = "node removed from the cluster" + } + return false, diff } + + // Sort by node IDs. sort.Slice(a, func(i, j int) bool { - return a[i] < a[j] + return a[i].id < a[j].id }) sort.Slice(b, func(i, j int) bool { - return b[i] < b[j] + return b[i].id < b[j].id }) - for i, v := range a { - if v != b[i] { - return false + for i := 0; i < len(a); i++ { + if a[i].id != b[i].id { + return false, "found different set of nodes" + } + if a[i].epoch != b[i].epoch { + return false, fmt.Sprintf("n%d was restarted", a[i].id) } } - return true + return true, "" +} + +func (ns nodes) String() string { + var b strings.Builder + b.WriteString("n{") + if len(ns) > 0 { + b.WriteString(fmt.Sprintf("%d", ns[0].id)) + for _, node := range ns[1:] { + b.WriteString(fmt.Sprintf(",%d", node.id)) + } + } + b.WriteString("}") + + return b.String() +} + +// SafeFormat implements redact.SafeFormatter. +func (ns nodes) SafeFormat(s redact.SafePrinter, _ rune) { + s.SafeString(redact.SafeString(ns.String())) } // fenceVersionFor constructs the appropriate "fence version" for the given diff --git a/pkg/migration/util_test.go b/pkg/migration/util_test.go new file mode 100644 index 000000000000..55b001ebd85f --- /dev/null +++ b/pkg/migration/util_test.go @@ -0,0 +1,87 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package migration + +import ( + "strconv" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" +) + +func TestNodesString(t *testing.T) { + defer leaktest.AfterTest(t) + + ns := func(ids ...int) nodes { + var nodes []node + for _, id := range ids { + nodes = append(nodes, node{id: roachpb.NodeID(id)}) + } + return nodes + } + + var tests = []struct { + ns nodes + exp string + }{ + {ns(), "n{}"}, + {ns(1), "n{1}"}, + {ns(1, 2, 3), "n{1,2,3}"}, + {ns(3, 4, 7), "n{3,4,7}"}, + } + + for _, test := range tests { + if got := test.ns.String(); got != test.exp { + t.Fatalf("expected %s, got %s", test.exp, got) + } + } +} + +func TestNodesIdentical(t *testing.T) { + defer leaktest.AfterTest(t) + + list := func(nodes ...string) nodes { // takes in strings of the form "id@epoch" + var ns []node + for _, n := range nodes { + parts := strings.Split(n, "@") + id, _ := strconv.Atoi(parts[0]) + epoch, _ := strconv.Atoi(parts[1]) + ns = append(ns, node{id: roachpb.NodeID(id), epoch: int64(epoch)}) + } + return ns + } + + var tests = []struct { + a, b nodes + expOk bool + expDiff string + }{ + {list(), list(), true, ""}, + {list("1@2"), list("1@2"), true, ""}, + {list("1@2"), list("1@3"), false, "n1 was restarted"}, + {list("3@2"), list("5@2"), false, "found different set of nodes"}, + {list("2@1", "1@2"), list("1@2", "2@1"), true, ""}, + {list("1@2"), list("1@2", "2@1"), false, "node added to the cluster"}, + {list("1@1", "2@1"), list("1@1"), false, "node removed from the cluster"}, + } + + for _, test := range tests { + ok, diff := test.a.identical(test.b) + if ok != test.expOk { + t.Fatalf("expected identical = %t, got %t", test.expOk, ok) + } + if diff != test.expDiff { + t.Fatalf("expected diff %q, got %q", test.expDiff, diff) + } + } +} diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 8432e765f221..93cb19d3b709 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1560,6 +1560,7 @@ func TestLint(t *testing.T) { if err := stream.ForEach( stream.Sequence( filter, + stream.GrepNot("migration/.*exported func TestingNewCluster returns unexported type"), stream.GrepNot("sql/.*exported func .* returns unexported type sql.planNode"), stream.GrepNot("pkg/sql/types/types.go.* var Uuid should be UUID"), stream.GrepNot("pkg/sql/oidext/oidext.go.*don't use underscores in Go names; const T_"), From 678a97e2d9077dbfa6ac1a64390cfada950319fd Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 7 Dec 2020 11:56:45 -0500 Subject: [PATCH 08/11] migration: parallelize execution of the EveryNode primitive Release note: None --- pkg/migration/BUILD.bazel | 1 + pkg/migration/helper.go | 29 +++++++++++++++++++---------- 2 files changed, 20 insertions(+), 10 deletions(-) diff --git a/pkg/migration/BUILD.bazel b/pkg/migration/BUILD.bazel index 633153491c15..022a6453bfa1 100644 --- a/pkg/migration/BUILD.bazel +++ b/pkg/migration/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/server/serverpb", "//pkg/sql", "//pkg/sql/sqlutil", + "//pkg/util/ctxgroup", "//pkg/util/log", "//vendor/github.com/cockroachdb/errors", "//vendor/github.com/cockroachdb/logtags", diff --git a/pkg/migration/helper.go b/pkg/migration/helper.go index 4a539073bb38..eab3dfe2b487 100644 --- a/pkg/migration/helper.go +++ b/pkg/migration/helper.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" "google.golang.org/grpc" @@ -109,17 +110,25 @@ func (h *Helper) EveryNode( } for { - // TODO(irfansharif): We can/should send out these RPCs in parallel. - log.Infof(ctx, "executing op=%s on nodes=%s", op, ns) + log.Infof(ctx, "executing %s on nodes %s", op, ns) + + grp := ctxgroup.WithContext(ctx) for _, node := range ns { - conn, err := h.c.dial(ctx, node.id) - if err != nil { - return err - } - client := serverpb.NewMigrationClient(conn) - if err := fn(ctx, client); err != nil { - return err - } + id := node.id // copy out of the loop variable + grp.GoCtx(func(ctx context.Context) error { + conn, err := h.c.dial(ctx, id) + if err != nil { + return err + } + client := serverpb.NewMigrationClient(conn) + if err := fn(ctx, client); err != nil { + return err + } + return nil + }) + } + if err := grp.Wait(); err != nil { + return err } curNodes, err := h.c.nodes(ctx) From d6cad63baddaac1d1955ab39e0cf93956cca90e9 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 7 Dec 2020 11:58:09 -0500 Subject: [PATCH 09/11] migration: introduce the IterateRangeDescriptors primitive It's not currently wired up to anything. We'll use it in future PRs to send out `Migrate` requests to the entire keyspace. This was originally prototyped in #57445. See the inline comments and the RFC (#48843) for the motivation here. Release note: None --- pkg/migration/BUILD.bazel | 10 ++++++ pkg/migration/client_test.go | 64 +++++++++++++++++++++++++++++++++ pkg/migration/helper.go | 68 ++++++++++++++++++++++++++++++++++++ pkg/migration/main_test.go | 29 +++++++++++++++ 4 files changed, 171 insertions(+) create mode 100644 pkg/migration/client_test.go create mode 100644 pkg/migration/main_test.go diff --git a/pkg/migration/BUILD.bazel b/pkg/migration/BUILD.bazel index 022a6453bfa1..1ade0c6f06ef 100644 --- a/pkg/migration/BUILD.bazel +++ b/pkg/migration/BUILD.bazel @@ -12,6 +12,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/clusterversion", + "//pkg/keys", "//pkg/kv", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", @@ -32,18 +33,27 @@ go_library( go_test( name = "migration_test", srcs = [ + "client_test.go", "helper_test.go", + "main_test.go", "util_test.go", ], embed = [":migration"], deps = [ "//pkg/clusterversion", "//pkg/kv", + "//pkg/kv/kvserver", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", "//pkg/server/serverpb", "//pkg/sql/sqlutil", + "//pkg/sql/tests", "//pkg/testutils", + "//pkg/testutils/serverutils", + "//pkg/testutils/testcluster", "//pkg/util/leaktest", "//pkg/util/syncutil", "//vendor/github.com/cockroachdb/errors", diff --git a/pkg/migration/client_test.go b/pkg/migration/client_test.go new file mode 100644 index 000000000000..a50494f662ba --- /dev/null +++ b/pkg/migration/client_test.go @@ -0,0 +1,64 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. +package migration_test + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/migration" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/sql/tests" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" +) + +func TestHelperIterateRangeDescriptors(t *testing.T) { + defer leaktest.AfterTest(t) + + cv := clusterversion.ClusterVersion{} + ctx := context.Background() + const numNodes = 1 + + params, _ := tests.CreateTestServerParams() + server, _, kvDB := serverutils.StartServer(t, params) + defer server.Stopper().Stop(context.Background()) + + var numRanges int + if err := server.GetStores().(*kvserver.Stores).VisitStores(func(s *kvserver.Store) error { + numRanges = s.ReplicaCount() + return nil + }); err != nil { + t.Fatal(err) + } + + c := migration.TestingNewCluster(numNodes, kvDB, server.InternalExecutor().(sqlutil.InternalExecutor)) + h := migration.TestingNewHelper(c, cv) + + for _, blockSize := range []int{1, 5, 10, 50} { + var numDescs int + init := func() { numDescs = 0 } + if err := h.IterateRangeDescriptors(ctx, blockSize, init, func(descriptors ...roachpb.RangeDescriptor) error { + numDescs += len(descriptors) + return nil + }); err != nil { + t.Fatal(err) + } + + // TODO(irfansharif): We always seem to include a second copy of r1's + // desc. Unsure why. + if numDescs != numRanges+1 { + t.Fatalf("expected to find %d ranges, found %d", numRanges+1, numDescs) + } + } +} diff --git a/pkg/migration/helper.go b/pkg/migration/helper.go index eab3dfe2b487..f285c9d905b2 100644 --- a/pkg/migration/helper.go +++ b/pkg/migration/helper.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -148,6 +149,73 @@ func (h *Helper) EveryNode( return nil } +// IterateRangeDescriptors provides a handle on every range descriptor in the +// system, which callers can then use to send out arbitrary KV requests to in +// order to run arbitrary KV-level migrations. These requests will typically +// just be the `Migrate` request, with code added within [1] to do the specific +// things intended for the specified version. +// +// It's important to note that the closure is being executed in the context of a +// distributed transaction that may be automatically retried. So something like +// the following is an anti-pattern: +// +// processed := 0 +// _ = h.IterateRangeDescriptors(..., +// func(descriptors ...roachpb.RangeDescriptor) error { +// processed += len(descriptors) // we'll over count if retried +// log.Infof(ctx, "processed %d ranges", processed) +// }, +// ) +// +// Instead we allow callers to pass in a callback to signal on every attempt +// (including the first). This lets us salvage the example above: +// +// var processed int +// init := func() { processed = 0 } +// _ = h.IterateRangeDescriptors(..., init, +// func(descriptors ...roachpb.RangeDescriptor) error { +// processed += len(descriptors) +// log.Infof(ctx, "processed %d ranges", processed) +// }, +// ) +// +// [1]: pkg/kv/kvserver/batch_eval/cmd_migrate.go +func (h *Helper) IterateRangeDescriptors( + ctx context.Context, blockSize int, init func(), fn func(...roachpb.RangeDescriptor) error, +) error { + if err := h.DB().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + // Inform the caller that we're starting a fresh attempt to page in + // range descriptors. + init() + + // Iterate through the meta ranges to pull out all the range descriptors. + return txn.Iterate(ctx, keys.MetaMin, keys.MetaMax, blockSize, + func(rows []kv.KeyValue) error { + descriptors := make([]roachpb.RangeDescriptor, len(rows)) + for i, row := range rows { + if err := row.ValueProto(&descriptors[i]); err != nil { + return errors.Wrapf(err, + "unable to unmarshal range descriptor from %s", + row.Key, + ) + } + } + + // Invoke fn with the current chunk (of size ~blockSize) of + // range descriptors. + if err := fn(descriptors...); err != nil { + return err + } + + return nil + }) + }); err != nil { + return err + } + + return nil +} + // DB provides exposes the underlying *kv.DB instance. func (h *Helper) DB() *kv.DB { return h.c.db() diff --git a/pkg/migration/main_test.go b/pkg/migration/main_test.go new file mode 100644 index 000000000000..cdc3f7742a5a --- /dev/null +++ b/pkg/migration/main_test.go @@ -0,0 +1,29 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package migration_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" +) + +func TestMain(m *testing.M) { + security.SetAssetLoader(securitytest.EmbeddedAssets) + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} From b8b772286dee5b374a0503f4ffecea16af9f6c52 Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 7 Dec 2020 15:09:08 -0500 Subject: [PATCH 10/11] *: introduce the kv ranged `Migrate` command This command forces all ranges overlapping with the request spans to execute the (below-raft) migrations corresponding to the specific, stated version. This has the effect of moving the range out of any legacy modes operation they may currently be in. KV waits for this command to durably apply on all the replicas before returning, guaranteeing to the caller that all pre-migration state has been completely purged from the system. We're currently not wiring it up to anything. We will in a future commit that introduces the truncated state migration. This commit was pulled out of our prototype in #57445. Release note: None --- pkg/kv/batch.go | 38 +- pkg/kv/db.go | 10 + pkg/kv/kvserver/batcheval/BUILD.bazel | 1 + pkg/kv/kvserver/batcheval/cmd_migrate.go | 60 + pkg/kv/kvserver/replica_write.go | 21 + pkg/roachpb/api.go | 10 + pkg/roachpb/api.pb.go | 2297 +++++++++++++--------- pkg/roachpb/api.proto | 22 + pkg/roachpb/batch_generated.go | 27 +- pkg/roachpb/method.go | 3 + pkg/roachpb/method_string.go | 19 +- pkg/ts/catalog/chart_catalog.go | 1 + 12 files changed, 1612 insertions(+), 897 deletions(-) create mode 100644 pkg/kv/kvserver/batcheval/cmd_migrate.go diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index 098d9336d619..b721d72b2d64 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -227,18 +227,10 @@ func (b *Batch) fillResults(ctx context.Context) { case *roachpb.DeleteRequest: row := &result.Rows[k] row.Key = []byte(args.(*roachpb.DeleteRequest).Key) - case *roachpb.DeleteRangeRequest: if result.Err == nil { result.Keys = reply.(*roachpb.DeleteRangeResponse).Keys } - - default: - if result.Err == nil { - result.Err = errors.Errorf("unsupported reply: %T for %T", - reply, args) - } - // Nothing to do for all methods below as they do not generate // any rows. case *roachpb.EndTxnRequest: @@ -264,6 +256,12 @@ func (b *Batch) fillResults(ctx context.Context) { case *roachpb.ImportRequest: case *roachpb.AdminScatterRequest: case *roachpb.AddSSTableRequest: + case *roachpb.MigrateRequest: + default: + if result.Err == nil { + result.Err = errors.Errorf("unsupported reply: %T for %T", + reply, args) + } } // Fill up the resume span. if result.Err == nil && reply != nil && reply.Header().ResumeSpan != nil { @@ -754,3 +752,27 @@ func (b *Batch) addSSTable( b.appendReqs(req) b.initResult(1, 0, notRaw, nil) } + +// migrate is only exported on DB. +//lint:ignore U1000 unused +func (b *Batch) migrate(s, e interface{}, version roachpb.Version) { + begin, err := marshalKey(s) + if err != nil { + b.initResult(0, 0, notRaw, err) + return + } + end, err := marshalKey(e) + if err != nil { + b.initResult(0, 0, notRaw, err) + return + } + req := &roachpb.MigrateRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: begin, + EndKey: end, + }, + Version: version, + } + b.appendReqs(req) + b.initResult(1, 0, notRaw, nil) +} diff --git a/pkg/kv/db.go b/pkg/kv/db.go index 89a2c001e695..ecbb729cf4a7 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -639,6 +639,16 @@ func (db *DB) AddSSTable( return getOneErr(db.Run(ctx, b), b) } +// Migrate proactively forces ranges overlapping with the provided keyspace to +// transition out of any legacy modes of operation (as defined by the target +// version). +//lint:ignore U1000 unused +func (db *DB) Migrate(ctx context.Context, begin, end interface{}, version roachpb.Version) error { + b := &Batch{} + b.migrate(begin, end, version) + return getOneErr(db.Run(ctx, b), b) +} + // sendAndFill is a helper which sends the given batch and fills its results, // returning the appropriate error which is either from the first failing call, // or an "internal" error. diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel index 77a5c3c86544..20f81184284b 100644 --- a/pkg/kv/kvserver/batcheval/BUILD.bazel +++ b/pkg/kv/kvserver/batcheval/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "cmd_lease_request.go", "cmd_lease_transfer.go", "cmd_merge.go", + "cmd_migrate.go", "cmd_push_txn.go", "cmd_put.go", "cmd_query_intent.go", diff --git a/pkg/kv/kvserver/batcheval/cmd_migrate.go b/pkg/kv/kvserver/batcheval/cmd_migrate.go new file mode 100644 index 000000000000..c7694f8110fe --- /dev/null +++ b/pkg/kv/kvserver/batcheval/cmd_migrate.go @@ -0,0 +1,60 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package batcheval + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/errors" +) + +func init() { + RegisterReadWriteCommand(roachpb.Migrate, declareKeysMigrate, Migrate) +} + +func declareKeysMigrate( + _ *roachpb.RangeDescriptor, _ roachpb.Header, _ roachpb.Request, _, _ *spanset.SpanSet, +) { +} + +// migrationRegistry is a global registry of all KV-level migrations. See +// pkg/migration for details around how the migrations defined here are +// wired up. +var migrationRegistry = make(map[roachpb.Version]migration) + +type migration func(context.Context, storage.ReadWriter, CommandArgs) (result.Result, error) + +func init() { + // registerMigration(clusterversion.WhateverMigration, whateverMigration) + _ = registerMigration +} + +func registerMigration(key clusterversion.Key, migration migration) { + migrationRegistry[clusterversion.ByKey(key)] = migration +} + +// Migrate executes the below-raft migration corresponding to the given version. +func Migrate( + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, _ roachpb.Response, +) (result.Result, error) { + args := cArgs.Args.(*roachpb.MigrateRequest) + + fn, ok := migrationRegistry[args.Version] + if !ok { + return result.Result{}, errors.Newf("migration for %s not found", args.Version) + } + return fn(ctx, readWriter, cArgs) +} diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index 1c8edfceec9e..7d3116945787 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -216,6 +216,27 @@ func (r *Replica) executeWriteBatch( log.Warningf(ctx, "%v", err) } } + if ba.Requests[0].GetMigrate() != nil && propResult.Err == nil { + // Migrate is special since it wants commands to be durably + // applied on all peers, which we achieve via waitForApplication. + // + // NB: We don't have to worry about extant snapshots creating + // replicas that start at an index before this Migrate request. + // Snapshots that don't include the recipient (as specified by + // replicaID and descriptor in the snap vs. the replicaID of the + // raft instance) are discarded by the recipient, and we're + // already checking against all replicas in the descriptor + // below. Snapshots are also discarded unless they move the LAI + // forward, so we're not worried about old snapshots (with + // indexes preceding the MLAI here) instantiating pre-migrated + // state in anyway. + desc := r.Desc() + // NB: waitForApplication already has a timeout. + applicationErr := waitForApplication( + ctx, r.store.cfg.NodeDialer, desc.RangeID, desc.Replicas().All(), + uint64(maxLeaseIndex)) + propResult.Err = roachpb.NewError(applicationErr) + } return propResult.Reply, nil, propResult.Err case <-slowTimer.C: slowTimer.Read = true diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 63a3b374477c..78c6546d2e12 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -711,6 +711,9 @@ func (*AdminScatterRequest) Method() Method { return AdminScatter } // Method implements the Request interface. func (*AddSSTableRequest) Method() Method { return AddSSTable } +// Method implements the Request interface. +func (*MigrateRequest) Method() Method { return Migrate } + // Method implements the Request interface. func (*RecomputeStatsRequest) Method() Method { return RecomputeStats } @@ -957,6 +960,12 @@ func (r *AddSSTableRequest) ShallowCopy() Request { return &shallowCopy } +// ShallowCopy implements the Request interface. +func (r *MigrateRequest) ShallowCopy() Request { + shallowCopy := *r + return &shallowCopy +} + // ShallowCopy implements the Request interface. func (r *RecomputeStatsRequest) ShallowCopy() Request { shallowCopy := *r @@ -1286,6 +1295,7 @@ func (*AdminVerifyProtectedTimestampRequest) flags() int { return isAdmin | isRa func (*AddSSTableRequest) flags() int { return isWrite | isRange | isAlone | isUnsplittable | canBackpressure } +func (*MigrateRequest) flags() int { return isWrite | isRange | isAlone } // RefreshRequest and RefreshRangeRequest both determine which timestamp cache // they update based on their Write parameter. diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 8806b751f919..635108069401 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -73,7 +73,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{0} + return fileDescriptor_api_0db44e1715aa2d9a, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -101,7 +101,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{1} + return fileDescriptor_api_0db44e1715aa2d9a, []int{1} } type ChecksumMode int32 @@ -148,7 +148,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{2} + return fileDescriptor_api_0db44e1715aa2d9a, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -179,7 +179,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{3} + return fileDescriptor_api_0db44e1715aa2d9a, []int{3} } type ExternalStorageProvider int32 @@ -220,7 +220,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{4} + return fileDescriptor_api_0db44e1715aa2d9a, []int{4} } type MVCCFilter int32 @@ -243,7 +243,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{5} + return fileDescriptor_api_0db44e1715aa2d9a, []int{5} } type ResponseHeader_ResumeReason int32 @@ -269,7 +269,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{1, 0} + return fileDescriptor_api_0db44e1715aa2d9a, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -311,7 +311,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{25, 0} + return fileDescriptor_api_0db44e1715aa2d9a, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -332,7 +332,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) String() string { return proto.CompactTextString(m) } func (*RequestHeader) ProtoMessage() {} func (*RequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{0} + return fileDescriptor_api_0db44e1715aa2d9a, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -403,7 +403,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } func (*ResponseHeader) ProtoMessage() {} func (*ResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{1} + return fileDescriptor_api_0db44e1715aa2d9a, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -437,7 +437,7 @@ func (m *GetRequest) Reset() { *m = GetRequest{} } func (m *GetRequest) String() string { return proto.CompactTextString(m) } func (*GetRequest) ProtoMessage() {} func (*GetRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{2} + return fileDescriptor_api_0db44e1715aa2d9a, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -480,7 +480,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} } func (m *GetResponse) String() string { return proto.CompactTextString(m) } func (*GetResponse) ProtoMessage() {} func (*GetResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{3} + return fileDescriptor_api_0db44e1715aa2d9a, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -523,7 +523,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} } func (m *PutRequest) String() string { return proto.CompactTextString(m) } func (*PutRequest) ProtoMessage() {} func (*PutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{4} + return fileDescriptor_api_0db44e1715aa2d9a, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -557,7 +557,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} } func (m *PutResponse) String() string { return proto.CompactTextString(m) } func (*PutResponse) ProtoMessage() {} func (*PutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{5} + return fileDescriptor_api_0db44e1715aa2d9a, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -639,7 +639,7 @@ func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} } func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) } func (*ConditionalPutRequest) ProtoMessage() {} func (*ConditionalPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{6} + return fileDescriptor_api_0db44e1715aa2d9a, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -674,7 +674,7 @@ func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{} func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) } func (*ConditionalPutResponse) ProtoMessage() {} func (*ConditionalPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{7} + return fileDescriptor_api_0db44e1715aa2d9a, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -720,7 +720,7 @@ func (m *InitPutRequest) Reset() { *m = InitPutRequest{} } func (m *InitPutRequest) String() string { return proto.CompactTextString(m) } func (*InitPutRequest) ProtoMessage() {} func (*InitPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{8} + return fileDescriptor_api_0db44e1715aa2d9a, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -754,7 +754,7 @@ func (m *InitPutResponse) Reset() { *m = InitPutResponse{} } func (m *InitPutResponse) String() string { return proto.CompactTextString(m) } func (*InitPutResponse) ProtoMessage() {} func (*InitPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{9} + return fileDescriptor_api_0db44e1715aa2d9a, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -794,7 +794,7 @@ func (m *IncrementRequest) Reset() { *m = IncrementRequest{} } func (m *IncrementRequest) String() string { return proto.CompactTextString(m) } func (*IncrementRequest) ProtoMessage() {} func (*IncrementRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{10} + return fileDescriptor_api_0db44e1715aa2d9a, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -831,7 +831,7 @@ func (m *IncrementResponse) Reset() { *m = IncrementResponse{} } func (m *IncrementResponse) String() string { return proto.CompactTextString(m) } func (*IncrementResponse) ProtoMessage() {} func (*IncrementResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{11} + return fileDescriptor_api_0db44e1715aa2d9a, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -865,7 +865,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRequest) ProtoMessage() {} func (*DeleteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{12} + return fileDescriptor_api_0db44e1715aa2d9a, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -899,7 +899,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} } func (m *DeleteResponse) String() string { return proto.CompactTextString(m) } func (*DeleteResponse) ProtoMessage() {} func (*DeleteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{13} + return fileDescriptor_api_0db44e1715aa2d9a, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -951,7 +951,7 @@ func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} } func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRangeRequest) ProtoMessage() {} func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{14} + return fileDescriptor_api_0db44e1715aa2d9a, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -988,7 +988,7 @@ func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} } func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) } func (*DeleteRangeResponse) ProtoMessage() {} func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{15} + return fileDescriptor_api_0db44e1715aa2d9a, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1043,7 +1043,7 @@ func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} } func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) } func (*ClearRangeRequest) ProtoMessage() {} func (*ClearRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{16} + return fileDescriptor_api_0db44e1715aa2d9a, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1077,7 +1077,7 @@ func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} } func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) } func (*ClearRangeResponse) ProtoMessage() {} func (*ClearRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{17} + return fileDescriptor_api_0db44e1715aa2d9a, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1118,7 +1118,7 @@ func (m *RevertRangeRequest) Reset() { *m = RevertRangeRequest{} } func (m *RevertRangeRequest) String() string { return proto.CompactTextString(m) } func (*RevertRangeRequest) ProtoMessage() {} func (*RevertRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{18} + return fileDescriptor_api_0db44e1715aa2d9a, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1152,7 +1152,7 @@ func (m *RevertRangeResponse) Reset() { *m = RevertRangeResponse{} } func (m *RevertRangeResponse) String() string { return proto.CompactTextString(m) } func (*RevertRangeResponse) ProtoMessage() {} func (*RevertRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{19} + return fileDescriptor_api_0db44e1715aa2d9a, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1203,7 +1203,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} } func (m *ScanRequest) String() string { return proto.CompactTextString(m) } func (*ScanRequest) ProtoMessage() {} func (*ScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{20} + return fileDescriptor_api_0db44e1715aa2d9a, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1255,7 +1255,7 @@ func (m *ScanResponse) Reset() { *m = ScanResponse{} } func (m *ScanResponse) String() string { return proto.CompactTextString(m) } func (*ScanResponse) ProtoMessage() {} func (*ScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{21} + return fileDescriptor_api_0db44e1715aa2d9a, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1306,7 +1306,7 @@ func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} } func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) } func (*ReverseScanRequest) ProtoMessage() {} func (*ReverseScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{22} + return fileDescriptor_api_0db44e1715aa2d9a, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1358,7 +1358,7 @@ func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} } func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) } func (*ReverseScanResponse) ProtoMessage() {} func (*ReverseScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{23} + return fileDescriptor_api_0db44e1715aa2d9a, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1411,7 +1411,7 @@ func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyRequest) ProtoMessage() {} func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{24} + return fileDescriptor_api_0db44e1715aa2d9a, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1448,7 +1448,7 @@ func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyRespon func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse) ProtoMessage() {} func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{25} + return fileDescriptor_api_0db44e1715aa2d9a, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1492,7 +1492,7 @@ func (m *CheckConsistencyResponse_Result) Reset() { *m = CheckConsistenc func (m *CheckConsistencyResponse_Result) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse_Result) ProtoMessage() {} func (*CheckConsistencyResponse_Result) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{25, 0} + return fileDescriptor_api_0db44e1715aa2d9a, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1540,7 +1540,7 @@ func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} } func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsRequest) ProtoMessage() {} func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{26} + return fileDescriptor_api_0db44e1715aa2d9a, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1576,7 +1576,7 @@ func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{} func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsResponse) ProtoMessage() {} func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{27} + return fileDescriptor_api_0db44e1715aa2d9a, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1687,7 +1687,7 @@ func (m *EndTxnRequest) Reset() { *m = EndTxnRequest{} } func (m *EndTxnRequest) String() string { return proto.CompactTextString(m) } func (*EndTxnRequest) ProtoMessage() {} func (*EndTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{28} + return fileDescriptor_api_0db44e1715aa2d9a, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1733,7 +1733,7 @@ func (m *EndTxnResponse) Reset() { *m = EndTxnResponse{} } func (m *EndTxnResponse) String() string { return proto.CompactTextString(m) } func (*EndTxnResponse) ProtoMessage() {} func (*EndTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{29} + return fileDescriptor_api_0db44e1715aa2d9a, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1794,7 +1794,7 @@ func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} } func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminSplitRequest) ProtoMessage() {} func (*AdminSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{30} + return fileDescriptor_api_0db44e1715aa2d9a, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1829,7 +1829,7 @@ func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} } func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminSplitResponse) ProtoMessage() {} func (*AdminSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{31} + return fileDescriptor_api_0db44e1715aa2d9a, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1868,7 +1868,7 @@ func (m *AdminUnsplitRequest) Reset() { *m = AdminUnsplitRequest{} } func (m *AdminUnsplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitRequest) ProtoMessage() {} func (*AdminUnsplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{32} + return fileDescriptor_api_0db44e1715aa2d9a, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1903,7 +1903,7 @@ func (m *AdminUnsplitResponse) Reset() { *m = AdminUnsplitResponse{} } func (m *AdminUnsplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitResponse) ProtoMessage() {} func (*AdminUnsplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{33} + return fileDescriptor_api_0db44e1715aa2d9a, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1946,7 +1946,7 @@ func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} } func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) } func (*AdminMergeRequest) ProtoMessage() {} func (*AdminMergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{34} + return fileDescriptor_api_0db44e1715aa2d9a, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1981,7 +1981,7 @@ func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} } func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) } func (*AdminMergeResponse) ProtoMessage() {} func (*AdminMergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{35} + return fileDescriptor_api_0db44e1715aa2d9a, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2019,7 +2019,7 @@ func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseReq func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseRequest) ProtoMessage() {} func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{36} + return fileDescriptor_api_0db44e1715aa2d9a, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2052,7 +2052,7 @@ func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseRe func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseResponse) ProtoMessage() {} func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{37} + return fileDescriptor_api_0db44e1715aa2d9a, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2087,7 +2087,7 @@ func (m *ReplicationChange) Reset() { *m = ReplicationChange{} } func (m *ReplicationChange) String() string { return proto.CompactTextString(m) } func (*ReplicationChange) ProtoMessage() {} func (*ReplicationChange) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{38} + return fileDescriptor_api_0db44e1715aa2d9a, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2145,7 +2145,7 @@ func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasR func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasRequest) ProtoMessage() {} func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{39} + return fileDescriptor_api_0db44e1715aa2d9a, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2180,7 +2180,7 @@ func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicas func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasResponse) ProtoMessage() {} func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{40} + return fileDescriptor_api_0db44e1715aa2d9a, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2217,7 +2217,7 @@ func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeReq func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeRequest) ProtoMessage() {} func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{41} + return fileDescriptor_api_0db44e1715aa2d9a, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2250,7 +2250,7 @@ func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeRe func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeResponse) ProtoMessage() {} func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{42} + return fileDescriptor_api_0db44e1715aa2d9a, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2289,7 +2289,7 @@ func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} } func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnRequest) ProtoMessage() {} func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{43} + return fileDescriptor_api_0db44e1715aa2d9a, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2326,7 +2326,7 @@ func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} } func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnResponse) ProtoMessage() {} func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{44} + return fileDescriptor_api_0db44e1715aa2d9a, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2364,7 +2364,7 @@ func (m *GCRequest) Reset() { *m = GCRequest{} } func (m *GCRequest) String() string { return proto.CompactTextString(m) } func (*GCRequest) ProtoMessage() {} func (*GCRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{45} + return fileDescriptor_api_0db44e1715aa2d9a, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2398,7 +2398,7 @@ func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} } func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) } func (*GCRequest_GCKey) ProtoMessage() {} func (*GCRequest_GCKey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{45, 0} + return fileDescriptor_api_0db44e1715aa2d9a, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2432,7 +2432,7 @@ func (m *GCResponse) Reset() { *m = GCResponse{} } func (m *GCResponse) String() string { return proto.CompactTextString(m) } func (*GCResponse) ProtoMessage() {} func (*GCResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{46} + return fileDescriptor_api_0db44e1715aa2d9a, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2501,7 +2501,7 @@ func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} } func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) } func (*PushTxnRequest) ProtoMessage() {} func (*PushTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{47} + return fileDescriptor_api_0db44e1715aa2d9a, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2544,7 +2544,7 @@ func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} } func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) } func (*PushTxnResponse) ProtoMessage() {} func (*PushTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{48} + return fileDescriptor_api_0db44e1715aa2d9a, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2591,7 +2591,7 @@ func (m *RecoverTxnRequest) Reset() { *m = RecoverTxnRequest{} } func (m *RecoverTxnRequest) String() string { return proto.CompactTextString(m) } func (*RecoverTxnRequest) ProtoMessage() {} func (*RecoverTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{49} + return fileDescriptor_api_0db44e1715aa2d9a, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2627,7 +2627,7 @@ func (m *RecoverTxnResponse) Reset() { *m = RecoverTxnResponse{} } func (m *RecoverTxnResponse) String() string { return proto.CompactTextString(m) } func (*RecoverTxnResponse) ProtoMessage() {} func (*RecoverTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{50} + return fileDescriptor_api_0db44e1715aa2d9a, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2671,7 +2671,7 @@ func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} } func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) } func (*QueryTxnRequest) ProtoMessage() {} func (*QueryTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{51} + return fileDescriptor_api_0db44e1715aa2d9a, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2715,7 +2715,7 @@ func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} } func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) } func (*QueryTxnResponse) ProtoMessage() {} func (*QueryTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{52} + return fileDescriptor_api_0db44e1715aa2d9a, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2775,7 +2775,7 @@ func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} } func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) } func (*QueryIntentRequest) ProtoMessage() {} func (*QueryIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{53} + return fileDescriptor_api_0db44e1715aa2d9a, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2811,7 +2811,7 @@ func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} } func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) } func (*QueryIntentResponse) ProtoMessage() {} func (*QueryIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{54} + return fileDescriptor_api_0db44e1715aa2d9a, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2858,7 +2858,7 @@ func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} } func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRequest) ProtoMessage() {} func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{55} + return fileDescriptor_api_0db44e1715aa2d9a, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2893,7 +2893,7 @@ func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} } func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentResponse) ProtoMessage() {} func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{56} + return fileDescriptor_api_0db44e1715aa2d9a, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2944,7 +2944,7 @@ func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeReq func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeRequest) ProtoMessage() {} func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{57} + return fileDescriptor_api_0db44e1715aa2d9a, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2979,7 +2979,7 @@ func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeRe func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeResponse) ProtoMessage() {} func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{58} + return fileDescriptor_api_0db44e1715aa2d9a, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3016,7 +3016,7 @@ func (m *MergeRequest) Reset() { *m = MergeRequest{} } func (m *MergeRequest) String() string { return proto.CompactTextString(m) } func (*MergeRequest) ProtoMessage() {} func (*MergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{59} + return fileDescriptor_api_0db44e1715aa2d9a, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3050,7 +3050,7 @@ func (m *MergeResponse) Reset() { *m = MergeResponse{} } func (m *MergeResponse) String() string { return proto.CompactTextString(m) } func (*MergeResponse) ProtoMessage() {} func (*MergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{60} + return fileDescriptor_api_0db44e1715aa2d9a, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3095,7 +3095,7 @@ func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} } func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) } func (*TruncateLogRequest) ProtoMessage() {} func (*TruncateLogRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{61} + return fileDescriptor_api_0db44e1715aa2d9a, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3129,7 +3129,7 @@ func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} } func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) } func (*TruncateLogResponse) ProtoMessage() {} func (*TruncateLogResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{62} + return fileDescriptor_api_0db44e1715aa2d9a, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3173,7 +3173,7 @@ func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} } func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) } func (*RequestLeaseRequest) ProtoMessage() {} func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{63} + return fileDescriptor_api_0db44e1715aa2d9a, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3222,7 +3222,7 @@ func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} } func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*TransferLeaseRequest) ProtoMessage() {} func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{64} + return fileDescriptor_api_0db44e1715aa2d9a, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3259,7 +3259,7 @@ func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} } func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) } func (*LeaseInfoRequest) ProtoMessage() {} func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{65} + return fileDescriptor_api_0db44e1715aa2d9a, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3296,7 +3296,7 @@ func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) } func (*LeaseInfoResponse) ProtoMessage() {} func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{66} + return fileDescriptor_api_0db44e1715aa2d9a, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3331,7 +3331,7 @@ func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} } func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) } func (*RequestLeaseResponse) ProtoMessage() {} func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{67} + return fileDescriptor_api_0db44e1715aa2d9a, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3386,7 +3386,7 @@ func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{} func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumRequest) ProtoMessage() {} func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{68} + return fileDescriptor_api_0db44e1715aa2d9a, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3423,7 +3423,7 @@ func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumResponse) ProtoMessage() {} func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{69} + return fileDescriptor_api_0db44e1715aa2d9a, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3463,7 +3463,7 @@ func (m *ExternalStorage) Reset() { *m = ExternalStorage{} } func (m *ExternalStorage) String() string { return proto.CompactTextString(m) } func (*ExternalStorage) ProtoMessage() {} func (*ExternalStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70} + return fileDescriptor_api_0db44e1715aa2d9a, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3497,7 +3497,7 @@ func (m *ExternalStorage_LocalFilePath) Reset() { *m = ExternalStorage_L func (m *ExternalStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_LocalFilePath) ProtoMessage() {} func (*ExternalStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70, 0} + return fileDescriptor_api_0db44e1715aa2d9a, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3530,7 +3530,7 @@ func (m *ExternalStorage_Http) Reset() { *m = ExternalStorage_Http{} } func (m *ExternalStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Http) ProtoMessage() {} func (*ExternalStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70, 1} + return fileDescriptor_api_0db44e1715aa2d9a, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3572,7 +3572,7 @@ func (m *ExternalStorage_S3) Reset() { *m = ExternalStorage_S3{} } func (m *ExternalStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_S3) ProtoMessage() {} func (*ExternalStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70, 2} + return fileDescriptor_api_0db44e1715aa2d9a, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3611,7 +3611,7 @@ func (m *ExternalStorage_GCS) Reset() { *m = ExternalStorage_GCS{} } func (m *ExternalStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_GCS) ProtoMessage() {} func (*ExternalStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70, 3} + return fileDescriptor_api_0db44e1715aa2d9a, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3647,7 +3647,7 @@ func (m *ExternalStorage_Azure) Reset() { *m = ExternalStorage_Azure{} } func (m *ExternalStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Azure) ProtoMessage() {} func (*ExternalStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70, 4} + return fileDescriptor_api_0db44e1715aa2d9a, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3686,7 +3686,7 @@ func (m *ExternalStorage_Workload) Reset() { *m = ExternalStorage_Worklo func (m *ExternalStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Workload) ProtoMessage() {} func (*ExternalStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70, 5} + return fileDescriptor_api_0db44e1715aa2d9a, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3728,7 +3728,7 @@ func (m *ExternalStorage_FileTable) Reset() { *m = ExternalStorage_FileT func (m *ExternalStorage_FileTable) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_FileTable) ProtoMessage() {} func (*ExternalStorage_FileTable) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70, 6} + return fileDescriptor_api_0db44e1715aa2d9a, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3768,7 +3768,7 @@ func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} } func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) } func (*WriteBatchRequest) ProtoMessage() {} func (*WriteBatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{71} + return fileDescriptor_api_0db44e1715aa2d9a, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3802,7 +3802,7 @@ func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} } func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) } func (*WriteBatchResponse) ProtoMessage() {} func (*WriteBatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{72} + return fileDescriptor_api_0db44e1715aa2d9a, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3838,7 +3838,7 @@ func (m *FileEncryptionOptions) Reset() { *m = FileEncryptionOptions{} } func (m *FileEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*FileEncryptionOptions) ProtoMessage() {} func (*FileEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{73} + return fileDescriptor_api_0db44e1715aa2d9a, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3908,7 +3908,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} } func (m *ExportRequest) String() string { return proto.CompactTextString(m) } func (*ExportRequest) ProtoMessage() {} func (*ExportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{74} + return fileDescriptor_api_0db44e1715aa2d9a, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3960,7 +3960,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } func (*BulkOpSummary) ProtoMessage() {} func (*BulkOpSummary) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{75} + return fileDescriptor_api_0db44e1715aa2d9a, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3996,7 +3996,7 @@ func (m *ExportResponse) Reset() { *m = ExportResponse{} } func (m *ExportResponse) String() string { return proto.CompactTextString(m) } func (*ExportResponse) ProtoMessage() {} func (*ExportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{76} + return fileDescriptor_api_0db44e1715aa2d9a, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4036,7 +4036,7 @@ func (m *ExportResponse_File) Reset() { *m = ExportResponse_File{} } func (m *ExportResponse_File) String() string { return proto.CompactTextString(m) } func (*ExportResponse_File) ProtoMessage() {} func (*ExportResponse_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{76, 0} + return fileDescriptor_api_0db44e1715aa2d9a, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4087,7 +4087,7 @@ func (m *ImportRequest) Reset() { *m = ImportRequest{} } func (m *ImportRequest) String() string { return proto.CompactTextString(m) } func (*ImportRequest) ProtoMessage() {} func (*ImportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{77} + return fileDescriptor_api_0db44e1715aa2d9a, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4122,7 +4122,7 @@ func (m *ImportRequest_File) Reset() { *m = ImportRequest_File{} } func (m *ImportRequest_File) String() string { return proto.CompactTextString(m) } func (*ImportRequest_File) ProtoMessage() {} func (*ImportRequest_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{77, 0} + return fileDescriptor_api_0db44e1715aa2d9a, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4158,7 +4158,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } func (*ImportRequest_TableRekey) ProtoMessage() {} func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{77, 1} + return fileDescriptor_api_0db44e1715aa2d9a, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4193,7 +4193,7 @@ func (m *ImportResponse) Reset() { *m = ImportResponse{} } func (m *ImportResponse) String() string { return proto.CompactTextString(m) } func (*ImportResponse) ProtoMessage() {} func (*ImportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{78} + return fileDescriptor_api_0db44e1715aa2d9a, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4231,7 +4231,7 @@ func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} } func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) } func (*AdminScatterRequest) ProtoMessage() {} func (*AdminScatterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{79} + return fileDescriptor_api_0db44e1715aa2d9a, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4268,7 +4268,7 @@ func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} } func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse) ProtoMessage() {} func (*AdminScatterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{80} + return fileDescriptor_api_0db44e1715aa2d9a, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4301,7 +4301,7 @@ func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse_Range) ProtoMessage() {} func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{80, 0} + return fileDescriptor_api_0db44e1715aa2d9a, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4346,7 +4346,7 @@ func (m *AdminVerifyProtectedTimestampRequest) Reset() { *m = AdminVerif func (m *AdminVerifyProtectedTimestampRequest) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampRequest) ProtoMessage() {} func (*AdminVerifyProtectedTimestampRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{81} + return fileDescriptor_api_0db44e1715aa2d9a, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4384,7 +4384,7 @@ func (m *AdminVerifyProtectedTimestampResponse) Reset() { *m = AdminVeri func (m *AdminVerifyProtectedTimestampResponse) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampResponse) ProtoMessage() {} func (*AdminVerifyProtectedTimestampResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{82} + return fileDescriptor_api_0db44e1715aa2d9a, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4437,7 +4437,7 @@ func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) } func (*AddSSTableRequest) ProtoMessage() {} func (*AddSSTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{83} + return fileDescriptor_api_0db44e1715aa2d9a, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4471,7 +4471,7 @@ func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} } func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) } func (*AddSSTableResponse) ProtoMessage() {} func (*AddSSTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{84} + return fileDescriptor_api_0db44e1715aa2d9a, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4515,7 +4515,7 @@ func (m *RefreshRequest) Reset() { *m = RefreshRequest{} } func (m *RefreshRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRequest) ProtoMessage() {} func (*RefreshRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{85} + return fileDescriptor_api_0db44e1715aa2d9a, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4549,7 +4549,7 @@ func (m *RefreshResponse) Reset() { *m = RefreshResponse{} } func (m *RefreshResponse) String() string { return proto.CompactTextString(m) } func (*RefreshResponse) ProtoMessage() {} func (*RefreshResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{86} + return fileDescriptor_api_0db44e1715aa2d9a, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4588,7 +4588,7 @@ func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} } func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRangeRequest) ProtoMessage() {} func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{87} + return fileDescriptor_api_0db44e1715aa2d9a, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4622,7 +4622,7 @@ func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} } func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) } func (*RefreshRangeResponse) ProtoMessage() {} func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{88} + return fileDescriptor_api_0db44e1715aa2d9a, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4671,7 +4671,7 @@ func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} } func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) } func (*SubsumeRequest) ProtoMessage() {} func (*SubsumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{89} + return fileDescriptor_api_0db44e1715aa2d9a, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4720,7 +4720,7 @@ func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } func (*SubsumeResponse) ProtoMessage() {} func (*SubsumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{90} + return fileDescriptor_api_0db44e1715aa2d9a, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4755,7 +4755,7 @@ func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} } func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RangeStatsRequest) ProtoMessage() {} func (*RangeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{91} + return fileDescriptor_api_0db44e1715aa2d9a, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4797,7 +4797,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RangeStatsResponse) ProtoMessage() {} func (*RangeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{92} + return fileDescriptor_api_0db44e1715aa2d9a, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4822,6 +4822,83 @@ func (m *RangeStatsResponse) XXX_DiscardUnknown() { var xxx_messageInfo_RangeStatsResponse proto.InternalMessageInfo +// MigrateRequest is used instruct all ranges overlapping with it to exercise +// any relevant (below-raft) migrations in order for its range state to conform +// to what's needed by the specified version. It's a core primitive used in our +// migrations infrastructure to phase out legacy code below raft. +// +// KV waits for this command to durably apply on all replicas before returning, +// guaranteeing to the caller that all pre-migration state has been completely +// purged from the system. +type MigrateRequest struct { + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + // Version is used to select the specific migration to exercise. + Version Version `protobuf:"bytes,2,opt,name=version,proto3" json:"version"` +} + +func (m *MigrateRequest) Reset() { *m = MigrateRequest{} } +func (m *MigrateRequest) String() string { return proto.CompactTextString(m) } +func (*MigrateRequest) ProtoMessage() {} +func (*MigrateRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_api_0db44e1715aa2d9a, []int{93} +} +func (m *MigrateRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MigrateRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MigrateRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_MigrateRequest.Merge(dst, src) +} +func (m *MigrateRequest) XXX_Size() int { + return m.Size() +} +func (m *MigrateRequest) XXX_DiscardUnknown() { + xxx_messageInfo_MigrateRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_MigrateRequest proto.InternalMessageInfo + +// MigrateResponse is the response to a Migrate operation. +type MigrateResponse struct { + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` +} + +func (m *MigrateResponse) Reset() { *m = MigrateResponse{} } +func (m *MigrateResponse) String() string { return proto.CompactTextString(m) } +func (*MigrateResponse) ProtoMessage() {} +func (*MigrateResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_api_0db44e1715aa2d9a, []int{94} +} +func (m *MigrateResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MigrateResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MigrateResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_MigrateResponse.Merge(dst, src) +} +func (m *MigrateResponse) XXX_Size() int { + return m.Size() +} +func (m *MigrateResponse) XXX_DiscardUnknown() { + xxx_messageInfo_MigrateResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_MigrateResponse proto.InternalMessageInfo + // A RequestUnion contains exactly one of the requests. // The values added here must match those in ResponseUnion. // @@ -4873,6 +4950,7 @@ type RequestUnion struct { // *RequestUnion_Subsume // *RequestUnion_RangeStats // *RequestUnion_AdminVerifyProtectedTimestamp + // *RequestUnion_Migrate Value isRequestUnion_Value `protobuf_oneof:"value"` } @@ -4880,7 +4958,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} } func (m *RequestUnion) String() string { return proto.CompactTextString(m) } func (*RequestUnion) ProtoMessage() {} func (*RequestUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{93} + return fileDescriptor_api_0db44e1715aa2d9a, []int{95} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5043,6 +5121,9 @@ type RequestUnion_RangeStats struct { type RequestUnion_AdminVerifyProtectedTimestamp struct { AdminVerifyProtectedTimestamp *AdminVerifyProtectedTimestampRequest `protobuf:"bytes,49,opt,name=admin_verify_protected_timestamp,json=adminVerifyProtectedTimestamp,proto3,oneof"` } +type RequestUnion_Migrate struct { + Migrate *MigrateRequest `protobuf:"bytes,50,opt,name=migrate,proto3,oneof"` +} func (*RequestUnion_Get) isRequestUnion_Value() {} func (*RequestUnion_Put) isRequestUnion_Value() {} @@ -5088,6 +5169,7 @@ func (*RequestUnion_RefreshRange) isRequestUnion_Value() {} func (*RequestUnion_Subsume) isRequestUnion_Value() {} func (*RequestUnion_RangeStats) isRequestUnion_Value() {} func (*RequestUnion_AdminVerifyProtectedTimestamp) isRequestUnion_Value() {} +func (*RequestUnion_Migrate) isRequestUnion_Value() {} func (m *RequestUnion) GetValue() isRequestUnion_Value { if m != nil { @@ -5404,6 +5486,13 @@ func (m *RequestUnion) GetAdminVerifyProtectedTimestamp() *AdminVerifyProtectedT return nil } +func (m *RequestUnion) GetMigrate() *MigrateRequest { + if x, ok := m.GetValue().(*RequestUnion_Migrate); ok { + return x.Migrate + } + return nil +} + // XXX_OneofFuncs is for the internal use of the proto package. func (*RequestUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { return _RequestUnion_OneofMarshaler, _RequestUnion_OneofUnmarshaler, _RequestUnion_OneofSizer, []interface{}{ @@ -5451,6 +5540,7 @@ func (*RequestUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) (*RequestUnion_Subsume)(nil), (*RequestUnion_RangeStats)(nil), (*RequestUnion_AdminVerifyProtectedTimestamp)(nil), + (*RequestUnion_Migrate)(nil), } } @@ -5678,6 +5768,11 @@ func _RequestUnion_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { if err := b.EncodeMessage(x.AdminVerifyProtectedTimestamp); err != nil { return err } + case *RequestUnion_Migrate: + _ = b.EncodeVarint(50<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Migrate); err != nil { + return err + } case nil: default: return fmt.Errorf("RequestUnion.Value has unexpected type %T", x) @@ -6040,6 +6135,14 @@ func _RequestUnion_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.B err := b.DecodeMessage(msg) m.Value = &RequestUnion_AdminVerifyProtectedTimestamp{msg} return true, err + case 50: // value.migrate + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(MigrateRequest) + err := b.DecodeMessage(msg) + m.Value = &RequestUnion_Migrate{msg} + return true, err default: return false, nil } @@ -6269,6 +6372,11 @@ func _RequestUnion_OneofSizer(msg proto.Message) (n int) { n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s + case *RequestUnion_Migrate: + s := proto.Size(x.Migrate) + n += 2 // tag and wire + n += proto.SizeVarint(uint64(s)) + n += s case nil: default: panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) @@ -6323,6 +6431,7 @@ type ResponseUnion struct { // *ResponseUnion_Subsume // *ResponseUnion_RangeStats // *ResponseUnion_AdminVerifyProtectedTimestamp + // *ResponseUnion_Migrate Value isResponseUnion_Value `protobuf_oneof:"value"` } @@ -6330,7 +6439,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } func (*ResponseUnion) ProtoMessage() {} func (*ResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{94} + return fileDescriptor_api_0db44e1715aa2d9a, []int{96} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6490,6 +6599,9 @@ type ResponseUnion_RangeStats struct { type ResponseUnion_AdminVerifyProtectedTimestamp struct { AdminVerifyProtectedTimestamp *AdminVerifyProtectedTimestampResponse `protobuf:"bytes,49,opt,name=admin_verify_protected_timestamp,json=adminVerifyProtectedTimestamp,proto3,oneof"` } +type ResponseUnion_Migrate struct { + Migrate *MigrateResponse `protobuf:"bytes,50,opt,name=migrate,proto3,oneof"` +} func (*ResponseUnion_Get) isResponseUnion_Value() {} func (*ResponseUnion_Put) isResponseUnion_Value() {} @@ -6534,6 +6646,7 @@ func (*ResponseUnion_RefreshRange) isResponseUnion_Value() {} func (*ResponseUnion_Subsume) isResponseUnion_Value() {} func (*ResponseUnion_RangeStats) isResponseUnion_Value() {} func (*ResponseUnion_AdminVerifyProtectedTimestamp) isResponseUnion_Value() {} +func (*ResponseUnion_Migrate) isResponseUnion_Value() {} func (m *ResponseUnion) GetValue() isResponseUnion_Value { if m != nil { @@ -6843,6 +6956,13 @@ func (m *ResponseUnion) GetAdminVerifyProtectedTimestamp() *AdminVerifyProtected return nil } +func (m *ResponseUnion) GetMigrate() *MigrateResponse { + if x, ok := m.GetValue().(*ResponseUnion_Migrate); ok { + return x.Migrate + } + return nil +} + // XXX_OneofFuncs is for the internal use of the proto package. func (*ResponseUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { return _ResponseUnion_OneofMarshaler, _ResponseUnion_OneofUnmarshaler, _ResponseUnion_OneofSizer, []interface{}{ @@ -6889,6 +7009,7 @@ func (*ResponseUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) (*ResponseUnion_Subsume)(nil), (*ResponseUnion_RangeStats)(nil), (*ResponseUnion_AdminVerifyProtectedTimestamp)(nil), + (*ResponseUnion_Migrate)(nil), } } @@ -7111,6 +7232,11 @@ func _ResponseUnion_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { if err := b.EncodeMessage(x.AdminVerifyProtectedTimestamp); err != nil { return err } + case *ResponseUnion_Migrate: + _ = b.EncodeVarint(50<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Migrate); err != nil { + return err + } case nil: default: return fmt.Errorf("ResponseUnion.Value has unexpected type %T", x) @@ -7465,6 +7591,14 @@ func _ResponseUnion_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto. err := b.DecodeMessage(msg) m.Value = &ResponseUnion_AdminVerifyProtectedTimestamp{msg} return true, err + case 50: // value.migrate + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(MigrateResponse) + err := b.DecodeMessage(msg) + m.Value = &ResponseUnion_Migrate{msg} + return true, err default: return false, nil } @@ -7689,6 +7823,11 @@ func _ResponseUnion_OneofSizer(msg proto.Message) (n int) { n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s + case *ResponseUnion_Migrate: + s := proto.Size(x.Migrate) + n += 2 // tag and wire + n += proto.SizeVarint(uint64(s)) + n += s case nil: default: panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) @@ -7865,7 +8004,7 @@ func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} func (*Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{95} + return fileDescriptor_api_0db44e1715aa2d9a, []int{97} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7903,7 +8042,7 @@ func (m *ClientRangeInfo) Reset() { *m = ClientRangeInfo{} } func (m *ClientRangeInfo) String() string { return proto.CompactTextString(m) } func (*ClientRangeInfo) ProtoMessage() {} func (*ClientRangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{96} + return fileDescriptor_api_0db44e1715aa2d9a, []int{98} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7939,7 +8078,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{97} + return fileDescriptor_api_0db44e1715aa2d9a, []int{99} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7976,7 +8115,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{98} + return fileDescriptor_api_0db44e1715aa2d9a, []int{100} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8055,7 +8194,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} } func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) } func (*BatchResponse_Header) ProtoMessage() {} func (*BatchResponse_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{98, 0} + return fileDescriptor_api_0db44e1715aa2d9a, []int{100, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8093,7 +8232,7 @@ func (m *RangeLookupRequest) Reset() { *m = RangeLookupRequest{} } func (m *RangeLookupRequest) String() string { return proto.CompactTextString(m) } func (*RangeLookupRequest) ProtoMessage() {} func (*RangeLookupRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{99} + return fileDescriptor_api_0db44e1715aa2d9a, []int{101} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8131,7 +8270,7 @@ func (m *RangeLookupResponse) Reset() { *m = RangeLookupResponse{} } func (m *RangeLookupResponse) String() string { return proto.CompactTextString(m) } func (*RangeLookupResponse) ProtoMessage() {} func (*RangeLookupResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{100} + return fileDescriptor_api_0db44e1715aa2d9a, []int{102} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8170,7 +8309,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} func (*RangeFeedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{101} + return fileDescriptor_api_0db44e1715aa2d9a, []int{103} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8211,7 +8350,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} func (*RangeFeedValue) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{102} + return fileDescriptor_api_0db44e1715aa2d9a, []int{104} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8252,7 +8391,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{103} + return fileDescriptor_api_0db44e1715aa2d9a, []int{105} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8289,7 +8428,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} func (*RangeFeedError) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{104} + return fileDescriptor_api_0db44e1715aa2d9a, []int{106} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8326,7 +8465,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} func (*RangeFeedEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{105} + return fileDescriptor_api_0db44e1715aa2d9a, []int{107} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8370,7 +8509,7 @@ func (m *ResetQuorumRequest) Reset() { *m = ResetQuorumRequest{} } func (m *ResetQuorumRequest) String() string { return proto.CompactTextString(m) } func (*ResetQuorumRequest) ProtoMessage() {} func (*ResetQuorumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{106} + return fileDescriptor_api_0db44e1715aa2d9a, []int{108} } func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8402,7 +8541,7 @@ func (m *ResetQuorumResponse) Reset() { *m = ResetQuorumResponse{} } func (m *ResetQuorumResponse) String() string { return proto.CompactTextString(m) } func (*ResetQuorumResponse) ProtoMessage() {} func (*ResetQuorumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{107} + return fileDescriptor_api_0db44e1715aa2d9a, []int{109} } func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8443,7 +8582,7 @@ func (m *GossipSubscriptionRequest) Reset() { *m = GossipSubscriptionReq func (m *GossipSubscriptionRequest) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionRequest) ProtoMessage() {} func (*GossipSubscriptionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{108} + return fileDescriptor_api_0db44e1715aa2d9a, []int{110} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8483,7 +8622,7 @@ func (m *GossipSubscriptionEvent) Reset() { *m = GossipSubscriptionEvent func (m *GossipSubscriptionEvent) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionEvent) ProtoMessage() {} func (*GossipSubscriptionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{109} + return fileDescriptor_api_0db44e1715aa2d9a, []int{111} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8519,7 +8658,7 @@ func (m *JoinNodeRequest) Reset() { *m = JoinNodeRequest{} } func (m *JoinNodeRequest) String() string { return proto.CompactTextString(m) } func (*JoinNodeRequest) ProtoMessage() {} func (*JoinNodeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{110} + return fileDescriptor_api_0db44e1715aa2d9a, []int{112} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8558,7 +8697,7 @@ func (m *JoinNodeResponse) Reset() { *m = JoinNodeResponse{} } func (m *JoinNodeResponse) String() string { return proto.CompactTextString(m) } func (*JoinNodeResponse) ProtoMessage() {} func (*JoinNodeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{111} + return fileDescriptor_api_0db44e1715aa2d9a, []int{113} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8601,7 +8740,7 @@ func (m *ContentionEvent) Reset() { *m = ContentionEvent{} } func (m *ContentionEvent) String() string { return proto.CompactTextString(m) } func (*ContentionEvent) ProtoMessage() {} func (*ContentionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{112} + return fileDescriptor_api_0db44e1715aa2d9a, []int{114} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8735,6 +8874,8 @@ func init() { proto.RegisterType((*SubsumeResponse)(nil), "cockroach.roachpb.SubsumeResponse") proto.RegisterType((*RangeStatsRequest)(nil), "cockroach.roachpb.RangeStatsRequest") proto.RegisterType((*RangeStatsResponse)(nil), "cockroach.roachpb.RangeStatsResponse") + proto.RegisterType((*MigrateRequest)(nil), "cockroach.roachpb.MigrateRequest") + proto.RegisterType((*MigrateResponse)(nil), "cockroach.roachpb.MigrateResponse") proto.RegisterType((*RequestUnion)(nil), "cockroach.roachpb.RequestUnion") proto.RegisterType((*ResponseUnion)(nil), "cockroach.roachpb.ResponseUnion") proto.RegisterType((*Header)(nil), "cockroach.roachpb.Header") @@ -13450,6 +13591,66 @@ func (m *RangeStatsResponse) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *MigrateRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *MigrateRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) + n162, err := m.RequestHeader.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n162 + dAtA[i] = 0x12 + i++ + i = encodeVarintApi(dAtA, i, uint64(m.Version.Size())) + n163, err := m.Version.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n163 + return i, nil +} + +func (m *MigrateResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *MigrateResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) + n164, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n164 + return i, nil +} + func (m *RequestUnion) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -13466,11 +13667,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Value != nil { - nn162, err := m.Value.MarshalTo(dAtA[i:]) + nn165, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn162 + i += nn165 } return i, nil } @@ -13481,11 +13682,11 @@ func (m *RequestUnion_Get) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Get.Size())) - n163, err := m.Get.MarshalTo(dAtA[i:]) + n166, err := m.Get.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n163 + i += n166 } return i, nil } @@ -13495,11 +13696,11 @@ func (m *RequestUnion_Put) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Put.Size())) - n164, err := m.Put.MarshalTo(dAtA[i:]) + n167, err := m.Put.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n164 + i += n167 } return i, nil } @@ -13509,11 +13710,11 @@ func (m *RequestUnion_ConditionalPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size())) - n165, err := m.ConditionalPut.MarshalTo(dAtA[i:]) + n168, err := m.ConditionalPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n165 + i += n168 } return i, nil } @@ -13523,11 +13724,11 @@ func (m *RequestUnion_Increment) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size())) - n166, err := m.Increment.MarshalTo(dAtA[i:]) + n169, err := m.Increment.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n166 + i += n169 } return i, nil } @@ -13537,11 +13738,11 @@ func (m *RequestUnion_Delete) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size())) - n167, err := m.Delete.MarshalTo(dAtA[i:]) + n170, err := m.Delete.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n167 + i += n170 } return i, nil } @@ -13551,11 +13752,11 @@ func (m *RequestUnion_DeleteRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size())) - n168, err := m.DeleteRange.MarshalTo(dAtA[i:]) + n171, err := m.DeleteRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n168 + i += n171 } return i, nil } @@ -13565,11 +13766,11 @@ func (m *RequestUnion_Scan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size())) - n169, err := m.Scan.MarshalTo(dAtA[i:]) + n172, err := m.Scan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n169 + i += n172 } return i, nil } @@ -13579,11 +13780,11 @@ func (m *RequestUnion_EndTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTxn.Size())) - n170, err := m.EndTxn.MarshalTo(dAtA[i:]) + n173, err := m.EndTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n170 + i += n173 } return i, nil } @@ -13593,11 +13794,11 @@ func (m *RequestUnion_AdminSplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size())) - n171, err := m.AdminSplit.MarshalTo(dAtA[i:]) + n174, err := m.AdminSplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n171 + i += n174 } return i, nil } @@ -13607,11 +13808,11 @@ func (m *RequestUnion_AdminMerge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size())) - n172, err := m.AdminMerge.MarshalTo(dAtA[i:]) + n175, err := m.AdminMerge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n172 + i += n175 } return i, nil } @@ -13621,11 +13822,11 @@ func (m *RequestUnion_HeartbeatTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size())) - n173, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) + n176, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n173 + i += n176 } return i, nil } @@ -13635,11 +13836,11 @@ func (m *RequestUnion_Gc) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size())) - n174, err := m.Gc.MarshalTo(dAtA[i:]) + n177, err := m.Gc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n174 + i += n177 } return i, nil } @@ -13649,11 +13850,11 @@ func (m *RequestUnion_PushTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size())) - n175, err := m.PushTxn.MarshalTo(dAtA[i:]) + n178, err := m.PushTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n175 + i += n178 } return i, nil } @@ -13665,11 +13866,11 @@ func (m *RequestUnion_ResolveIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size())) - n176, err := m.ResolveIntent.MarshalTo(dAtA[i:]) + n179, err := m.ResolveIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n176 + i += n179 } return i, nil } @@ -13681,11 +13882,11 @@ func (m *RequestUnion_ResolveIntentRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size())) - n177, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) + n180, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n177 + i += n180 } return i, nil } @@ -13697,11 +13898,11 @@ func (m *RequestUnion_Merge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size())) - n178, err := m.Merge.MarshalTo(dAtA[i:]) + n181, err := m.Merge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n178 + i += n181 } return i, nil } @@ -13713,11 +13914,11 @@ func (m *RequestUnion_TruncateLog) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size())) - n179, err := m.TruncateLog.MarshalTo(dAtA[i:]) + n182, err := m.TruncateLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n179 + i += n182 } return i, nil } @@ -13729,11 +13930,11 @@ func (m *RequestUnion_RequestLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size())) - n180, err := m.RequestLease.MarshalTo(dAtA[i:]) + n183, err := m.RequestLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n180 + i += n183 } return i, nil } @@ -13745,11 +13946,11 @@ func (m *RequestUnion_ReverseScan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size())) - n181, err := m.ReverseScan.MarshalTo(dAtA[i:]) + n184, err := m.ReverseScan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n181 + i += n184 } return i, nil } @@ -13761,11 +13962,11 @@ func (m *RequestUnion_ComputeChecksum) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size())) - n182, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) + n185, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n182 + i += n185 } return i, nil } @@ -13777,11 +13978,11 @@ func (m *RequestUnion_CheckConsistency) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size())) - n183, err := m.CheckConsistency.MarshalTo(dAtA[i:]) + n186, err := m.CheckConsistency.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n183 + i += n186 } return i, nil } @@ -13793,11 +13994,11 @@ func (m *RequestUnion_InitPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size())) - n184, err := m.InitPut.MarshalTo(dAtA[i:]) + n187, err := m.InitPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n184 + i += n187 } return i, nil } @@ -13809,11 +14010,11 @@ func (m *RequestUnion_TransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TransferLease.Size())) - n185, err := m.TransferLease.MarshalTo(dAtA[i:]) + n188, err := m.TransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n185 + i += n188 } return i, nil } @@ -13825,11 +14026,11 @@ func (m *RequestUnion_AdminTransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size())) - n186, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) + n189, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n186 + i += n189 } return i, nil } @@ -13841,11 +14042,11 @@ func (m *RequestUnion_LeaseInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size())) - n187, err := m.LeaseInfo.MarshalTo(dAtA[i:]) + n190, err := m.LeaseInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n187 + i += n190 } return i, nil } @@ -13857,11 +14058,11 @@ func (m *RequestUnion_WriteBatch) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size())) - n188, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n191, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n188 + i += n191 } return i, nil } @@ -13873,11 +14074,11 @@ func (m *RequestUnion_Export) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Export.Size())) - n189, err := m.Export.MarshalTo(dAtA[i:]) + n192, err := m.Export.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n189 + i += n192 } return i, nil } @@ -13889,11 +14090,11 @@ func (m *RequestUnion_QueryTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size())) - n190, err := m.QueryTxn.MarshalTo(dAtA[i:]) + n193, err := m.QueryTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n190 + i += n193 } return i, nil } @@ -13905,11 +14106,11 @@ func (m *RequestUnion_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Import.Size())) - n191, err := m.Import.MarshalTo(dAtA[i:]) + n194, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n191 + i += n194 } return i, nil } @@ -13921,11 +14122,11 @@ func (m *RequestUnion_AdminChangeReplicas) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size())) - n192, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) + n195, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n192 + i += n195 } return i, nil } @@ -13937,11 +14138,11 @@ func (m *RequestUnion_AdminScatter) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size())) - n193, err := m.AdminScatter.MarshalTo(dAtA[i:]) + n196, err := m.AdminScatter.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n193 + i += n196 } return i, nil } @@ -13953,11 +14154,11 @@ func (m *RequestUnion_AddSstable) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size())) - n194, err := m.AddSstable.MarshalTo(dAtA[i:]) + n197, err := m.AddSstable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n194 + i += n197 } return i, nil } @@ -13969,11 +14170,11 @@ func (m *RequestUnion_ClearRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClearRange.Size())) - n195, err := m.ClearRange.MarshalTo(dAtA[i:]) + n198, err := m.ClearRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n195 + i += n198 } return i, nil } @@ -13985,11 +14186,11 @@ func (m *RequestUnion_RecomputeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecomputeStats.Size())) - n196, err := m.RecomputeStats.MarshalTo(dAtA[i:]) + n199, err := m.RecomputeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n196 + i += n199 } return i, nil } @@ -14001,11 +14202,11 @@ func (m *RequestUnion_Refresh) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Refresh.Size())) - n197, err := m.Refresh.MarshalTo(dAtA[i:]) + n200, err := m.Refresh.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n197 + i += n200 } return i, nil } @@ -14017,11 +14218,11 @@ func (m *RequestUnion_RefreshRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RefreshRange.Size())) - n198, err := m.RefreshRange.MarshalTo(dAtA[i:]) + n201, err := m.RefreshRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n198 + i += n201 } return i, nil } @@ -14033,11 +14234,11 @@ func (m *RequestUnion_QueryIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryIntent.Size())) - n199, err := m.QueryIntent.MarshalTo(dAtA[i:]) + n202, err := m.QueryIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n199 + i += n202 } return i, nil } @@ -14049,11 +14250,11 @@ func (m *RequestUnion_Subsume) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Subsume.Size())) - n200, err := m.Subsume.MarshalTo(dAtA[i:]) + n203, err := m.Subsume.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n200 + i += n203 } return i, nil } @@ -14065,11 +14266,11 @@ func (m *RequestUnion_RangeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeStats.Size())) - n201, err := m.RangeStats.MarshalTo(dAtA[i:]) + n204, err := m.RangeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n201 + i += n204 } return i, nil } @@ -14081,11 +14282,11 @@ func (m *RequestUnion_AdminRelocateRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminRelocateRange.Size())) - n202, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) + n205, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n202 + i += n205 } return i, nil } @@ -14097,11 +14298,11 @@ func (m *RequestUnion_RecoverTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecoverTxn.Size())) - n203, err := m.RecoverTxn.MarshalTo(dAtA[i:]) + n206, err := m.RecoverTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n203 + i += n206 } return i, nil } @@ -14113,11 +14314,11 @@ func (m *RequestUnion_AdminUnsplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminUnsplit.Size())) - n204, err := m.AdminUnsplit.MarshalTo(dAtA[i:]) + n207, err := m.AdminUnsplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n204 + i += n207 } return i, nil } @@ -14129,11 +14330,11 @@ func (m *RequestUnion_RevertRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3 i++ i = encodeVarintApi(dAtA, i, uint64(m.RevertRange.Size())) - n205, err := m.RevertRange.MarshalTo(dAtA[i:]) + n208, err := m.RevertRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n205 + i += n208 } return i, nil } @@ -14145,11 +14346,27 @@ func (m *RequestUnion_AdminVerifyProtectedTimestamp) MarshalTo(dAtA []byte) (int dAtA[i] = 0x3 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminVerifyProtectedTimestamp.Size())) - n206, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:]) + n209, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n206 + i += n209 + } + return i, nil +} +func (m *RequestUnion_Migrate) MarshalTo(dAtA []byte) (int, error) { + i := 0 + if m.Migrate != nil { + dAtA[i] = 0x92 + i++ + dAtA[i] = 0x3 + i++ + i = encodeVarintApi(dAtA, i, uint64(m.Migrate.Size())) + n210, err := m.Migrate.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n210 } return i, nil } @@ -14169,11 +14386,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Value != nil { - nn207, err := m.Value.MarshalTo(dAtA[i:]) + nn211, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn207 + i += nn211 } return i, nil } @@ -14184,11 +14401,11 @@ func (m *ResponseUnion_Get) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Get.Size())) - n208, err := m.Get.MarshalTo(dAtA[i:]) + n212, err := m.Get.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n208 + i += n212 } return i, nil } @@ -14198,11 +14415,11 @@ func (m *ResponseUnion_Put) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Put.Size())) - n209, err := m.Put.MarshalTo(dAtA[i:]) + n213, err := m.Put.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n209 + i += n213 } return i, nil } @@ -14212,11 +14429,11 @@ func (m *ResponseUnion_ConditionalPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size())) - n210, err := m.ConditionalPut.MarshalTo(dAtA[i:]) + n214, err := m.ConditionalPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n210 + i += n214 } return i, nil } @@ -14226,11 +14443,11 @@ func (m *ResponseUnion_Increment) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size())) - n211, err := m.Increment.MarshalTo(dAtA[i:]) + n215, err := m.Increment.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n211 + i += n215 } return i, nil } @@ -14240,11 +14457,11 @@ func (m *ResponseUnion_Delete) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size())) - n212, err := m.Delete.MarshalTo(dAtA[i:]) + n216, err := m.Delete.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n212 + i += n216 } return i, nil } @@ -14254,11 +14471,11 @@ func (m *ResponseUnion_DeleteRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size())) - n213, err := m.DeleteRange.MarshalTo(dAtA[i:]) + n217, err := m.DeleteRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n213 + i += n217 } return i, nil } @@ -14268,11 +14485,11 @@ func (m *ResponseUnion_Scan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size())) - n214, err := m.Scan.MarshalTo(dAtA[i:]) + n218, err := m.Scan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n214 + i += n218 } return i, nil } @@ -14282,11 +14499,11 @@ func (m *ResponseUnion_EndTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTxn.Size())) - n215, err := m.EndTxn.MarshalTo(dAtA[i:]) + n219, err := m.EndTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n215 + i += n219 } return i, nil } @@ -14296,11 +14513,11 @@ func (m *ResponseUnion_AdminSplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size())) - n216, err := m.AdminSplit.MarshalTo(dAtA[i:]) + n220, err := m.AdminSplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n216 + i += n220 } return i, nil } @@ -14310,11 +14527,11 @@ func (m *ResponseUnion_AdminMerge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size())) - n217, err := m.AdminMerge.MarshalTo(dAtA[i:]) + n221, err := m.AdminMerge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n217 + i += n221 } return i, nil } @@ -14324,11 +14541,11 @@ func (m *ResponseUnion_HeartbeatTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size())) - n218, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) + n222, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n218 + i += n222 } return i, nil } @@ -14338,11 +14555,11 @@ func (m *ResponseUnion_Gc) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size())) - n219, err := m.Gc.MarshalTo(dAtA[i:]) + n223, err := m.Gc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n219 + i += n223 } return i, nil } @@ -14352,11 +14569,11 @@ func (m *ResponseUnion_PushTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size())) - n220, err := m.PushTxn.MarshalTo(dAtA[i:]) + n224, err := m.PushTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n220 + i += n224 } return i, nil } @@ -14368,11 +14585,11 @@ func (m *ResponseUnion_ResolveIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size())) - n221, err := m.ResolveIntent.MarshalTo(dAtA[i:]) + n225, err := m.ResolveIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n221 + i += n225 } return i, nil } @@ -14384,11 +14601,11 @@ func (m *ResponseUnion_ResolveIntentRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size())) - n222, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) + n226, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n222 + i += n226 } return i, nil } @@ -14400,11 +14617,11 @@ func (m *ResponseUnion_Merge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size())) - n223, err := m.Merge.MarshalTo(dAtA[i:]) + n227, err := m.Merge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n223 + i += n227 } return i, nil } @@ -14416,11 +14633,11 @@ func (m *ResponseUnion_TruncateLog) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size())) - n224, err := m.TruncateLog.MarshalTo(dAtA[i:]) + n228, err := m.TruncateLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n224 + i += n228 } return i, nil } @@ -14432,11 +14649,11 @@ func (m *ResponseUnion_RequestLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size())) - n225, err := m.RequestLease.MarshalTo(dAtA[i:]) + n229, err := m.RequestLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n225 + i += n229 } return i, nil } @@ -14448,11 +14665,11 @@ func (m *ResponseUnion_ReverseScan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size())) - n226, err := m.ReverseScan.MarshalTo(dAtA[i:]) + n230, err := m.ReverseScan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n226 + i += n230 } return i, nil } @@ -14464,11 +14681,11 @@ func (m *ResponseUnion_ComputeChecksum) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size())) - n227, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) + n231, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n227 + i += n231 } return i, nil } @@ -14480,11 +14697,11 @@ func (m *ResponseUnion_CheckConsistency) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size())) - n228, err := m.CheckConsistency.MarshalTo(dAtA[i:]) + n232, err := m.CheckConsistency.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n228 + i += n232 } return i, nil } @@ -14496,11 +14713,11 @@ func (m *ResponseUnion_InitPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size())) - n229, err := m.InitPut.MarshalTo(dAtA[i:]) + n233, err := m.InitPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n229 + i += n233 } return i, nil } @@ -14512,11 +14729,11 @@ func (m *ResponseUnion_AdminTransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size())) - n230, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) + n234, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n230 + i += n234 } return i, nil } @@ -14528,11 +14745,11 @@ func (m *ResponseUnion_LeaseInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size())) - n231, err := m.LeaseInfo.MarshalTo(dAtA[i:]) + n235, err := m.LeaseInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n231 + i += n235 } return i, nil } @@ -14544,11 +14761,11 @@ func (m *ResponseUnion_WriteBatch) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size())) - n232, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n236, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n232 + i += n236 } return i, nil } @@ -14560,11 +14777,11 @@ func (m *ResponseUnion_Export) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Export.Size())) - n233, err := m.Export.MarshalTo(dAtA[i:]) + n237, err := m.Export.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n233 + i += n237 } return i, nil } @@ -14576,11 +14793,11 @@ func (m *ResponseUnion_QueryTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size())) - n234, err := m.QueryTxn.MarshalTo(dAtA[i:]) + n238, err := m.QueryTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n234 + i += n238 } return i, nil } @@ -14592,11 +14809,11 @@ func (m *ResponseUnion_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Import.Size())) - n235, err := m.Import.MarshalTo(dAtA[i:]) + n239, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n235 + i += n239 } return i, nil } @@ -14608,11 +14825,11 @@ func (m *ResponseUnion_AdminChangeReplicas) MarshalTo(dAtA []byte) (int, error) dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size())) - n236, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) + n240, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n236 + i += n240 } return i, nil } @@ -14624,11 +14841,11 @@ func (m *ResponseUnion_AdminScatter) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size())) - n237, err := m.AdminScatter.MarshalTo(dAtA[i:]) + n241, err := m.AdminScatter.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n237 + i += n241 } return i, nil } @@ -14640,11 +14857,11 @@ func (m *ResponseUnion_AddSstable) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size())) - n238, err := m.AddSstable.MarshalTo(dAtA[i:]) + n242, err := m.AddSstable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n238 + i += n242 } return i, nil } @@ -14656,11 +14873,11 @@ func (m *ResponseUnion_ClearRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClearRange.Size())) - n239, err := m.ClearRange.MarshalTo(dAtA[i:]) + n243, err := m.ClearRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n239 + i += n243 } return i, nil } @@ -14672,11 +14889,11 @@ func (m *ResponseUnion_RecomputeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecomputeStats.Size())) - n240, err := m.RecomputeStats.MarshalTo(dAtA[i:]) + n244, err := m.RecomputeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n240 + i += n244 } return i, nil } @@ -14688,11 +14905,11 @@ func (m *ResponseUnion_Refresh) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Refresh.Size())) - n241, err := m.Refresh.MarshalTo(dAtA[i:]) + n245, err := m.Refresh.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n241 + i += n245 } return i, nil } @@ -14704,11 +14921,11 @@ func (m *ResponseUnion_RefreshRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RefreshRange.Size())) - n242, err := m.RefreshRange.MarshalTo(dAtA[i:]) + n246, err := m.RefreshRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n242 + i += n246 } return i, nil } @@ -14720,11 +14937,11 @@ func (m *ResponseUnion_QueryIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryIntent.Size())) - n243, err := m.QueryIntent.MarshalTo(dAtA[i:]) + n247, err := m.QueryIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n243 + i += n247 } return i, nil } @@ -14736,11 +14953,11 @@ func (m *ResponseUnion_Subsume) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Subsume.Size())) - n244, err := m.Subsume.MarshalTo(dAtA[i:]) + n248, err := m.Subsume.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n244 + i += n248 } return i, nil } @@ -14752,11 +14969,11 @@ func (m *ResponseUnion_RangeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeStats.Size())) - n245, err := m.RangeStats.MarshalTo(dAtA[i:]) + n249, err := m.RangeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n245 + i += n249 } return i, nil } @@ -14768,11 +14985,11 @@ func (m *ResponseUnion_AdminRelocateRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminRelocateRange.Size())) - n246, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) + n250, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n246 + i += n250 } return i, nil } @@ -14784,11 +15001,11 @@ func (m *ResponseUnion_RecoverTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecoverTxn.Size())) - n247, err := m.RecoverTxn.MarshalTo(dAtA[i:]) + n251, err := m.RecoverTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n247 + i += n251 } return i, nil } @@ -14800,11 +15017,11 @@ func (m *ResponseUnion_AdminUnsplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminUnsplit.Size())) - n248, err := m.AdminUnsplit.MarshalTo(dAtA[i:]) + n252, err := m.AdminUnsplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n248 + i += n252 } return i, nil } @@ -14816,11 +15033,11 @@ func (m *ResponseUnion_RevertRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3 i++ i = encodeVarintApi(dAtA, i, uint64(m.RevertRange.Size())) - n249, err := m.RevertRange.MarshalTo(dAtA[i:]) + n253, err := m.RevertRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n249 + i += n253 } return i, nil } @@ -14832,11 +15049,27 @@ func (m *ResponseUnion_AdminVerifyProtectedTimestamp) MarshalTo(dAtA []byte) (in dAtA[i] = 0x3 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminVerifyProtectedTimestamp.Size())) - n250, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:]) + n254, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n250 + i += n254 + } + return i, nil +} +func (m *ResponseUnion_Migrate) MarshalTo(dAtA []byte) (int, error) { + i := 0 + if m.Migrate != nil { + dAtA[i] = 0x92 + i++ + dAtA[i] = 0x3 + i++ + i = encodeVarintApi(dAtA, i, uint64(m.Migrate.Size())) + n255, err := m.Migrate.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n255 } return i, nil } @@ -14858,19 +15091,19 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n251, err := m.Timestamp.MarshalTo(dAtA[i:]) + n256, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n251 + i += n256 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Replica.Size())) - n252, err := m.Replica.MarshalTo(dAtA[i:]) + n257, err := m.Replica.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n252 + i += n257 if m.RangeID != 0 { dAtA[i] = 0x18 i++ @@ -14886,11 +15119,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n253, err := m.Txn.MarshalTo(dAtA[i:]) + n258, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n253 + i += n258 } if m.ReadConsistency != 0 { dAtA[i] = 0x30 @@ -14960,11 +15193,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClientRangeInfo.Size())) - n254, err := m.ClientRangeInfo.MarshalTo(dAtA[i:]) + n259, err := m.ClientRangeInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n254 + i += n259 } if m.WaitPolicy != 0 { dAtA[i] = 0x90 @@ -15022,11 +15255,11 @@ func (m *BatchRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Header.Size())) - n255, err := m.Header.MarshalTo(dAtA[i:]) + n260, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n255 + i += n260 if len(m.Requests) > 0 { for _, msg := range m.Requests { dAtA[i] = 0x12 @@ -15060,11 +15293,11 @@ func (m *BatchResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.BatchResponse_Header.Size())) - n256, err := m.BatchResponse_Header.MarshalTo(dAtA[i:]) + n261, err := m.BatchResponse_Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n256 + i += n261 if len(m.Responses) > 0 { for _, msg := range m.Responses { dAtA[i] = 0x12 @@ -15099,38 +15332,38 @@ func (m *BatchResponse_Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n257, err := m.Error.MarshalTo(dAtA[i:]) + n262, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n257 + i += n262 } dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n258, err := m.Timestamp.MarshalTo(dAtA[i:]) + n263, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n258 + i += n263 if m.Txn != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n259, err := m.Txn.MarshalTo(dAtA[i:]) + n264, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n259 + i += n264 } dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Now.Size())) - n260, err := m.Now.MarshalTo(dAtA[i:]) + n265, err := m.Now.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n260 + i += n265 if len(m.CollectedSpans) > 0 { for _, msg := range m.CollectedSpans { dAtA[i] = 0x32 @@ -15245,11 +15478,11 @@ func (m *RangeLookupResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n261, err := m.Error.MarshalTo(dAtA[i:]) + n266, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n261 + i += n266 } return i, nil } @@ -15272,19 +15505,19 @@ func (m *RangeFeedRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Header.Size())) - n262, err := m.Header.MarshalTo(dAtA[i:]) + n267, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n262 + i += n267 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n263, err := m.Span.MarshalTo(dAtA[i:]) + n268, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n263 + i += n268 if m.WithDiff { dAtA[i] = 0x18 i++ @@ -15322,19 +15555,19 @@ func (m *RangeFeedValue) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Value.Size())) - n264, err := m.Value.MarshalTo(dAtA[i:]) + n269, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n264 + i += n269 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.PrevValue.Size())) - n265, err := m.PrevValue.MarshalTo(dAtA[i:]) + n270, err := m.PrevValue.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n265 + i += n270 return i, nil } @@ -15356,19 +15589,19 @@ func (m *RangeFeedCheckpoint) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n266, err := m.Span.MarshalTo(dAtA[i:]) + n271, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n266 + i += n271 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolvedTS.Size())) - n267, err := m.ResolvedTS.MarshalTo(dAtA[i:]) + n272, err := m.ResolvedTS.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n267 + i += n272 return i, nil } @@ -15390,11 +15623,11 @@ func (m *RangeFeedError) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n268, err := m.Error.MarshalTo(dAtA[i:]) + n273, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n268 + i += n273 return i, nil } @@ -15417,31 +15650,31 @@ func (m *RangeFeedEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Val.Size())) - n269, err := m.Val.MarshalTo(dAtA[i:]) + n274, err := m.Val.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n269 + i += n274 } if m.Checkpoint != nil { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Checkpoint.Size())) - n270, err := m.Checkpoint.MarshalTo(dAtA[i:]) + n275, err := m.Checkpoint.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n270 + i += n275 } if m.Error != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n271, err := m.Error.MarshalTo(dAtA[i:]) + n276, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n271 + i += n276 } return i, nil } @@ -15544,11 +15777,11 @@ func (m *GossipSubscriptionEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Content.Size())) - n272, err := m.Content.MarshalTo(dAtA[i:]) + n277, err := m.Content.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n272 + i += n277 if len(m.PatternMatched) > 0 { dAtA[i] = 0x1a i++ @@ -15559,11 +15792,11 @@ func (m *GossipSubscriptionEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n273, err := m.Error.MarshalTo(dAtA[i:]) + n278, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n273 + i += n278 } return i, nil } @@ -15587,11 +15820,11 @@ func (m *JoinNodeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.BinaryVersion.Size())) - n274, err := m.BinaryVersion.MarshalTo(dAtA[i:]) + n279, err := m.BinaryVersion.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n274 + i += n279 } return i, nil } @@ -15631,11 +15864,11 @@ func (m *JoinNodeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.ActiveVersion.Size())) - n275, err := m.ActiveVersion.MarshalTo(dAtA[i:]) + n280, err := m.ActiveVersion.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n275 + i += n280 } return i, nil } @@ -15664,19 +15897,19 @@ func (m *ContentionEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n276, err := m.Txn.MarshalTo(dAtA[i:]) + n281, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n276 + i += n281 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration))) - n277, err := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i:]) + n282, err := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i:]) if err != nil { return 0, err } - i += n277 + i += n282 return i, nil } @@ -17571,6 +17804,30 @@ func (m *RangeStatsResponse) Size() (n int) { return n } +func (m *MigrateRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.RequestHeader.Size() + n += 1 + l + sovApi(uint64(l)) + l = m.Version.Size() + n += 1 + l + sovApi(uint64(l)) + return n +} + +func (m *MigrateResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.ResponseHeader.Size() + n += 1 + l + sovApi(uint64(l)) + return n +} + func (m *RequestUnion) Size() (n int) { if m == nil { return 0 @@ -18111,6 +18368,18 @@ func (m *RequestUnion_AdminVerifyProtectedTimestamp) Size() (n int) { } return n } +func (m *RequestUnion_Migrate) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Migrate != nil { + l = m.Migrate.Size() + n += 2 + l + sovApi(uint64(l)) + } + return n +} func (m *ResponseUnion) Size() (n int) { if m == nil { return 0 @@ -18639,6 +18908,18 @@ func (m *ResponseUnion_AdminVerifyProtectedTimestamp) Size() (n int) { } return n } +func (m *ResponseUnion_Migrate) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Migrate != nil { + l = m.Migrate.Size() + n += 2 + l + sovApi(uint64(l)) + } + return n +} func (m *Header) Size() (n int) { if m == nil { return 0 @@ -32948,6 +33229,196 @@ func (m *RangeStatsResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *MigrateRequest) Unmarshal(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 ErrIntOverflowApi + } + 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: MigrateRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MigrateRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RequestHeader", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.RequestHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Version.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *MigrateResponse) Unmarshal(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 ErrIntOverflowApi + } + 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: MigrateResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: MigrateResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ResponseHeader", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.ResponseHeader.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipApi(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthApi + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *RequestUnion) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -34385,6 +34856,38 @@ func (m *RequestUnion) Unmarshal(dAtA []byte) error { } m.Value = &RequestUnion_AdminVerifyProtectedTimestamp{v} iNdEx = postIndex + case 50: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Migrate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &MigrateRequest{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &RequestUnion_Migrate{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -35811,6 +36314,38 @@ func (m *ResponseUnion) Unmarshal(dAtA []byte) error { } m.Value = &ResponseUnion_AdminVerifyProtectedTimestamp{v} iNdEx = postIndex + case 50: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Migrate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &MigrateResponse{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &ResponseUnion_Migrate{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -38505,516 +39040,520 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_290a2f113c3e6c17) } - -var fileDescriptor_api_290a2f113c3e6c17 = []byte{ - // 8124 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0x59, - 0x76, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0x50, 0xfc, 0xd1, 0x95, 0xba, 0x9b, 0xad, 0x99, 0x69, 0xa9, - 0xab, 0xff, 0x7b, 0x67, 0xa4, 0xe9, 0xee, 0x1d, 0xcf, 0x78, 0x7a, 0x3c, 0x6b, 0x91, 0x62, 0x37, - 0x29, 0xb5, 0xd4, 0xea, 0x22, 0xd5, 0xed, 0x19, 0xaf, 0x53, 0x5b, 0xaa, 0xba, 0xa2, 0x6a, 0x45, - 0x56, 0xb1, 0xab, 0x8a, 0xfa, 0x19, 0x20, 0x40, 0x1c, 0x1b, 0x89, 0x81, 0x00, 0x0b, 0x3f, 0x38, - 0xc8, 0x1a, 0x4e, 0xe2, 0x75, 0x1c, 0xc7, 0x01, 0x82, 0x20, 0x01, 0x12, 0xc4, 0x41, 0x90, 0xd8, - 0x2f, 0x46, 0xb2, 0x08, 0x0c, 0x64, 0xfd, 0x14, 0x23, 0x40, 0x14, 0x5b, 0x9b, 0xb7, 0x04, 0x46, - 0x90, 0x97, 0x00, 0xf3, 0x10, 0x04, 0xf7, 0xa7, 0xfe, 0xc8, 0x22, 0x45, 0xf5, 0xd6, 0x20, 0x03, - 0xec, 0x8b, 0xc4, 0x3a, 0x75, 0xcf, 0xa9, 0x7b, 0xcf, 0xbd, 0xf7, 0xdc, 0xf3, 0xdd, 0x3a, 0xf7, - 0x14, 0xcc, 0x5a, 0xa6, 0xa2, 0xee, 0x77, 0x77, 0x57, 0x94, 0xae, 0xbe, 0xdc, 0xb5, 0x4c, 0xc7, - 0x44, 0xb3, 0xaa, 0xa9, 0x1e, 0x50, 0xf2, 0x32, 0xbf, 0xb9, 0x70, 0xff, 0xe0, 0x70, 0xe5, 0xe0, - 0xd0, 0xc6, 0xd6, 0x21, 0xb6, 0x56, 0x54, 0xd3, 0x50, 0x7b, 0x96, 0x85, 0x0d, 0xf5, 0x64, 0xa5, - 0x6d, 0xaa, 0x07, 0xf4, 0x8f, 0x6e, 0xb4, 0x18, 0xfb, 0x02, 0x72, 0x25, 0x6a, 0x8a, 0xa3, 0x70, - 0xda, 0xbc, 0x4b, 0xc3, 0x96, 0x65, 0x5a, 0x36, 0xa7, 0x5e, 0x76, 0xa9, 0x1d, 0xec, 0x28, 0x81, - 0xd2, 0x6f, 0xd9, 0x8e, 0x69, 0x29, 0x2d, 0xbc, 0x82, 0x8d, 0x96, 0x6e, 0x60, 0x52, 0xe0, 0x50, - 0x55, 0xf9, 0xcd, 0xb7, 0x23, 0x6f, 0x3e, 0xe2, 0x77, 0x4b, 0x3d, 0x47, 0x6f, 0xaf, 0xec, 0xb7, - 0xd5, 0x15, 0x47, 0xef, 0x60, 0xdb, 0x51, 0x3a, 0x5d, 0x7e, 0xe7, 0x3e, 0xbd, 0xe3, 0x58, 0x8a, - 0xaa, 0x1b, 0x2d, 0xf7, 0x7f, 0x77, 0x77, 0xc5, 0xc2, 0xaa, 0x69, 0x69, 0x58, 0x93, 0xed, 0xae, - 0x62, 0xb8, 0xd5, 0x6d, 0x99, 0x2d, 0x93, 0xfe, 0x5c, 0x21, 0xbf, 0x38, 0xf5, 0x5a, 0xcb, 0x34, - 0x5b, 0x6d, 0xbc, 0x42, 0xaf, 0x76, 0x7b, 0x7b, 0x2b, 0x5a, 0xcf, 0x52, 0x1c, 0xdd, 0xe4, 0x5c, - 0xe2, 0xbf, 0x12, 0x20, 0x27, 0xe1, 0xd7, 0x3d, 0x6c, 0x3b, 0x35, 0xac, 0x68, 0xd8, 0x42, 0x57, - 0x21, 0x79, 0x80, 0x4f, 0x4a, 0xc9, 0x25, 0xe1, 0xee, 0x4c, 0x79, 0xfa, 0xcb, 0xd3, 0xc5, 0xe4, - 0x06, 0x3e, 0x91, 0x08, 0x0d, 0x2d, 0xc1, 0x34, 0x36, 0x34, 0x99, 0xdc, 0x4e, 0x85, 0x6f, 0x4f, - 0x61, 0x43, 0xdb, 0xc0, 0x27, 0xe8, 0xdb, 0x90, 0xb6, 0x89, 0x34, 0x43, 0xc5, 0xa5, 0xc9, 0x25, - 0xe1, 0xee, 0x64, 0xf9, 0xe7, 0xbf, 0x3c, 0x5d, 0xfc, 0xa4, 0xa5, 0x3b, 0xfb, 0xbd, 0xdd, 0x65, - 0xd5, 0xec, 0xac, 0x78, 0xfd, 0xa4, 0xed, 0xfa, 0xbf, 0x57, 0xba, 0x07, 0xad, 0x95, 0x7e, 0x1d, - 0x2d, 0x37, 0x8f, 0x8d, 0x06, 0x7e, 0x2d, 0x79, 0x12, 0xd7, 0x53, 0x69, 0xa1, 0x98, 0x58, 0x4f, - 0xa5, 0x13, 0xc5, 0xa4, 0xf8, 0xbb, 0x49, 0xc8, 0x4b, 0xd8, 0xee, 0x9a, 0x86, 0x8d, 0x79, 0xcd, - 0xdf, 0x87, 0xa4, 0x73, 0x6c, 0xd0, 0x9a, 0x67, 0x1f, 0x5e, 0x5b, 0x1e, 0x18, 0x11, 0xcb, 0x4d, - 0x4b, 0x31, 0x6c, 0x45, 0x25, 0xcd, 0x97, 0x48, 0x51, 0xf4, 0x11, 0x64, 0x2d, 0x6c, 0xf7, 0x3a, - 0x98, 0x2a, 0x92, 0x36, 0x2a, 0xfb, 0xf0, 0x4a, 0x04, 0x67, 0xa3, 0xab, 0x18, 0x12, 0xb0, 0xb2, - 0xe4, 0x37, 0xba, 0x0a, 0x69, 0xa3, 0xd7, 0x21, 0xaa, 0xb0, 0x69, 0x43, 0x93, 0xd2, 0xb4, 0xd1, - 0xeb, 0x6c, 0xe0, 0x13, 0x1b, 0xfd, 0x02, 0x5c, 0xd6, 0x70, 0xd7, 0xc2, 0xaa, 0xe2, 0x60, 0x4d, - 0xb6, 0x14, 0xa3, 0x85, 0x65, 0xdd, 0xd8, 0x33, 0xed, 0xd2, 0xd4, 0x52, 0xf2, 0x6e, 0xf6, 0xe1, - 0xdb, 0x11, 0xf2, 0x25, 0x52, 0xaa, 0x6e, 0xec, 0x99, 0xe5, 0xd4, 0x0f, 0x4f, 0x17, 0x27, 0xa4, - 0x79, 0x5f, 0x82, 0x77, 0xcb, 0x46, 0x0d, 0xc8, 0xf1, 0xea, 0x5a, 0x58, 0xb1, 0x4d, 0xa3, 0x34, - 0xbd, 0x24, 0xdc, 0xcd, 0x3f, 0x5c, 0x8e, 0x12, 0x18, 0x52, 0x0d, 0xb9, 0xec, 0x75, 0xb0, 0x44, - 0xb9, 0xa4, 0x19, 0x2b, 0x70, 0x85, 0xde, 0x82, 0x0c, 0x69, 0xc9, 0xee, 0x89, 0x83, 0xed, 0x52, - 0x9a, 0x36, 0x85, 0x34, 0xad, 0x4c, 0xae, 0xc5, 0x4f, 0x61, 0x26, 0xc8, 0x8a, 0x10, 0xe4, 0xa5, - 0x6a, 0x63, 0x67, 0xb3, 0x2a, 0xef, 0x6c, 0x6d, 0x6c, 0x3d, 0x7f, 0xb5, 0x55, 0x9c, 0x40, 0xf3, - 0x50, 0xe4, 0xb4, 0x8d, 0xea, 0x67, 0xf2, 0xb3, 0xfa, 0x66, 0xbd, 0x59, 0x14, 0x16, 0x52, 0xbf, - 0xf6, 0xbb, 0xd7, 0x26, 0xc4, 0x6d, 0x80, 0xa7, 0xd8, 0xe1, 0x03, 0x0c, 0x95, 0x61, 0x6a, 0x9f, - 0xd6, 0xa7, 0x24, 0x50, 0x4d, 0x2f, 0x45, 0x56, 0x3c, 0x30, 0x18, 0xcb, 0x69, 0xa2, 0x8d, 0x1f, - 0x9d, 0x2e, 0x0a, 0x12, 0xe7, 0x14, 0xff, 0x48, 0x80, 0x2c, 0x15, 0xc9, 0xda, 0x87, 0x2a, 0x7d, - 0x32, 0xaf, 0x9f, 0xab, 0x8c, 0x41, 0xa1, 0x68, 0x19, 0x26, 0x0f, 0x95, 0x76, 0x0f, 0x97, 0x12, - 0x54, 0x46, 0x29, 0x42, 0xc6, 0x4b, 0x72, 0x5f, 0x62, 0xc5, 0xd0, 0x63, 0x98, 0xd1, 0x0d, 0x07, - 0x1b, 0x8e, 0xcc, 0xd8, 0x92, 0xe7, 0xb0, 0x65, 0x59, 0x69, 0x7a, 0x21, 0xfe, 0x4b, 0x01, 0x60, - 0xbb, 0x17, 0xa7, 0x52, 0xd0, 0x37, 0xc7, 0xac, 0x3f, 0x1f, 0x5d, 0xbc, 0x15, 0x97, 0x61, 0x4a, - 0x37, 0xda, 0xba, 0xc1, 0xea, 0x9f, 0x96, 0xf8, 0x15, 0x9a, 0x87, 0xc9, 0xdd, 0xb6, 0x6e, 0x68, - 0x74, 0x3e, 0xa4, 0x25, 0x76, 0x21, 0x4a, 0x90, 0xa5, 0xb5, 0x8e, 0x51, 0xef, 0xe2, 0xbf, 0x4f, - 0xc0, 0xa5, 0x8a, 0x69, 0x68, 0x3a, 0x99, 0x92, 0x4a, 0xfb, 0x6b, 0xa1, 0x95, 0x75, 0x08, 0x4c, - 0x3e, 0x19, 0x1f, 0x77, 0xc7, 0xec, 0x63, 0xe4, 0x73, 0x55, 0x8f, 0xbb, 0x94, 0x16, 0xad, 0x49, - 0xf4, 0x4d, 0xb8, 0xa2, 0xb4, 0xdb, 0xe6, 0x91, 0xac, 0xef, 0xc9, 0x9a, 0x89, 0x6d, 0xd9, 0x30, - 0x1d, 0x19, 0x1f, 0xeb, 0xb6, 0x43, 0x4d, 0x49, 0x5a, 0x9a, 0xa3, 0xb7, 0xeb, 0x7b, 0x6b, 0x26, - 0xb6, 0xb7, 0x4c, 0xa7, 0x4a, 0x6e, 0x91, 0x79, 0x4a, 0x2a, 0xc3, 0xe6, 0xe9, 0x14, 0x31, 0xbf, - 0x52, 0x1a, 0x1f, 0x77, 0xd9, 0x3c, 0xfd, 0x25, 0xb8, 0xdc, 0xaf, 0xc7, 0x38, 0xfb, 0xe9, 0x4f, - 0x04, 0xc8, 0xd7, 0x0d, 0xdd, 0xf9, 0x5a, 0x74, 0x90, 0xa7, 0xd4, 0x64, 0x50, 0xa9, 0xf7, 0xa1, - 0xb8, 0xa7, 0xe8, 0xed, 0xe7, 0x46, 0xd3, 0xec, 0xec, 0xda, 0x8e, 0x69, 0x60, 0x9b, 0x6b, 0x7d, - 0x80, 0x2e, 0xbe, 0x84, 0x82, 0xd7, 0x9a, 0x38, 0xd5, 0xe4, 0x40, 0xb1, 0x6e, 0xa8, 0x16, 0xee, - 0x60, 0x23, 0x56, 0x3d, 0xbd, 0x0d, 0x19, 0xdd, 0x95, 0x4b, 0x75, 0x95, 0x94, 0x7c, 0x82, 0xd8, - 0x83, 0xd9, 0xc0, 0x53, 0xe3, 0x34, 0x8b, 0x64, 0x69, 0xc0, 0x47, 0xb2, 0xdf, 0x47, 0x64, 0x69, - 0xc0, 0x47, 0xcc, 0x8c, 0x35, 0x20, 0xb7, 0x86, 0xdb, 0xd8, 0xc1, 0x71, 0x5a, 0xf7, 0x1d, 0xc8, - 0xbb, 0x42, 0xe3, 0xec, 0x98, 0xbf, 0x23, 0x00, 0xe2, 0x72, 0xc9, 0x6a, 0x1a, 0x67, 0xdf, 0x2c, - 0x12, 0x17, 0xc2, 0xe9, 0x59, 0x06, 0xf3, 0x05, 0xd8, 0x98, 0x04, 0x46, 0xa2, 0xee, 0x80, 0x6f, - 0x65, 0x53, 0x41, 0x2b, 0xcb, 0xdd, 0x98, 0x23, 0x98, 0x0b, 0x55, 0x2c, 0xde, 0xee, 0x4b, 0xd1, - 0x3a, 0x25, 0x96, 0x92, 0x41, 0x5f, 0x8d, 0x12, 0xc5, 0xef, 0x0b, 0x30, 0x5b, 0x69, 0x63, 0xc5, - 0x8a, 0x5d, 0x23, 0xdf, 0x82, 0xb4, 0x86, 0x15, 0x8d, 0x36, 0x99, 0x4d, 0xec, 0x77, 0x02, 0x52, - 0x88, 0x47, 0xbb, 0xbc, 0xdf, 0x56, 0x97, 0x9b, 0xae, 0xaf, 0xcb, 0x67, 0xb7, 0xc7, 0x24, 0x7e, - 0x06, 0x28, 0x58, 0xb3, 0x38, 0x07, 0xc2, 0xdf, 0x17, 0x00, 0x49, 0xf8, 0x10, 0x5b, 0x4e, 0xec, - 0xcd, 0x5e, 0x83, 0xac, 0xa3, 0x58, 0x2d, 0xec, 0xc8, 0xc4, 0x8b, 0xbf, 0x48, 0xcb, 0x81, 0xf1, - 0x11, 0xb2, 0xf8, 0x39, 0xcc, 0x85, 0xea, 0x17, 0x67, 0xe3, 0xff, 0x97, 0x00, 0xd9, 0x86, 0xaa, - 0x18, 0x71, 0xb6, 0xfa, 0x53, 0xc8, 0xda, 0xaa, 0x62, 0xc8, 0x7b, 0xa6, 0xd5, 0x51, 0x1c, 0x3a, - 0xc4, 0xf3, 0xa1, 0x56, 0x7b, 0x1e, 0xb4, 0xaa, 0x18, 0x4f, 0x68, 0x21, 0x09, 0x6c, 0xef, 0x37, - 0x7a, 0x01, 0xd9, 0x03, 0x7c, 0x22, 0x73, 0x34, 0x46, 0xd7, 0xbf, 0xfc, 0xc3, 0xf7, 0x03, 0xfc, - 0x07, 0x87, 0xcb, 0x2e, 0x88, 0x5b, 0x0e, 0x80, 0xb8, 0x65, 0xc2, 0xb1, 0xdc, 0x70, 0x2c, 0x6c, - 0xb4, 0x9c, 0x7d, 0x09, 0x0e, 0xf0, 0xc9, 0x33, 0x26, 0x83, 0x4d, 0xac, 0xf5, 0x54, 0x3a, 0x59, - 0x4c, 0x89, 0xff, 0x47, 0x80, 0x19, 0xd6, 0xe4, 0x38, 0x27, 0xd6, 0x07, 0x90, 0xb2, 0xcc, 0x23, - 0x36, 0xb1, 0xb2, 0x0f, 0xdf, 0x8a, 0x10, 0xb1, 0x81, 0x4f, 0x82, 0x2b, 0x17, 0x2d, 0x8e, 0xca, - 0xc0, 0xfd, 0x40, 0x99, 0x72, 0x27, 0xc7, 0xe5, 0x06, 0xc6, 0x25, 0x11, 0x19, 0x77, 0xa0, 0xb0, - 0xab, 0x38, 0xea, 0xbe, 0x6c, 0xf1, 0x4a, 0x92, 0x55, 0x2e, 0x79, 0x77, 0x46, 0xca, 0x53, 0xb2, - 0x5b, 0x75, 0x9b, 0xb4, 0x9c, 0x8d, 0x74, 0x1b, 0xff, 0x94, 0xf5, 0xf9, 0xff, 0x15, 0xf8, 0x1c, - 0x72, 0x5b, 0xfe, 0xd3, 0xd6, 0xf5, 0xbf, 0x99, 0x80, 0x2b, 0x95, 0x7d, 0xac, 0x1e, 0x54, 0x4c, - 0xc3, 0xd6, 0x6d, 0x87, 0xe8, 0x2e, 0xce, 0xfe, 0x7f, 0x0b, 0x32, 0x47, 0xba, 0xb3, 0x2f, 0x6b, - 0xfa, 0xde, 0x1e, 0xb5, 0x73, 0x69, 0x29, 0x4d, 0x08, 0x6b, 0xfa, 0xde, 0x1e, 0x7a, 0x04, 0xa9, - 0x8e, 0xa9, 0x31, 0x77, 0x39, 0xff, 0x70, 0x31, 0x42, 0x3c, 0xad, 0x9a, 0xdd, 0xeb, 0x6c, 0x9a, - 0x1a, 0x96, 0x68, 0x61, 0x74, 0x0d, 0x40, 0x25, 0xd4, 0xae, 0xa9, 0x1b, 0x0e, 0x5f, 0x27, 0x03, - 0x14, 0x54, 0x83, 0x8c, 0x83, 0xad, 0x8e, 0x6e, 0x28, 0x0e, 0x2e, 0x4d, 0x52, 0xe5, 0xdd, 0x8c, - 0xac, 0x78, 0xb7, 0xad, 0xab, 0xca, 0x1a, 0xb6, 0x55, 0x4b, 0xef, 0x3a, 0xa6, 0xc5, 0xb5, 0xe8, - 0x33, 0x8b, 0xdf, 0x4b, 0x41, 0x69, 0x50, 0x37, 0x71, 0x8e, 0x90, 0x6d, 0x98, 0x22, 0xf8, 0xba, - 0xed, 0xf0, 0x31, 0xf2, 0x70, 0x98, 0x0a, 0x22, 0x6a, 0x40, 0x71, 0x7a, 0xdb, 0xe1, 0xd5, 0xe6, - 0x72, 0x16, 0xfe, 0xad, 0x00, 0x53, 0xec, 0x06, 0x7a, 0x00, 0x69, 0xbe, 0xa1, 0xa0, 0xd1, 0x3a, - 0x26, 0xcb, 0x97, 0xcf, 0x4e, 0x17, 0xa7, 0xd9, 0x1e, 0xc1, 0xda, 0x97, 0xfe, 0x4f, 0x69, 0x9a, - 0x96, 0xab, 0x6b, 0xa4, 0xb7, 0x6c, 0x47, 0xb1, 0x1c, 0xba, 0x6d, 0x93, 0x60, 0xb8, 0x81, 0x12, - 0x36, 0xf0, 0x09, 0x5a, 0x87, 0x29, 0xdb, 0x51, 0x9c, 0x9e, 0xcd, 0xfb, 0xeb, 0x42, 0x95, 0x6d, - 0x50, 0x4e, 0x89, 0x4b, 0x20, 0x8e, 0x8e, 0x86, 0x1d, 0x45, 0x6f, 0xd3, 0x0e, 0xcc, 0x48, 0xfc, - 0x4a, 0xfc, 0x2d, 0x01, 0xa6, 0x58, 0x51, 0x74, 0x05, 0xe6, 0xa4, 0xd5, 0xad, 0xa7, 0x55, 0xb9, - 0xbe, 0xb5, 0x56, 0x6d, 0x56, 0xa5, 0xcd, 0xfa, 0xd6, 0x6a, 0xb3, 0x5a, 0x9c, 0x40, 0x97, 0x01, - 0xb9, 0x37, 0x2a, 0xcf, 0xb7, 0x1a, 0xf5, 0x46, 0xb3, 0xba, 0xd5, 0x2c, 0x0a, 0x74, 0x6f, 0x81, - 0xd2, 0x03, 0xd4, 0x04, 0xba, 0x09, 0x4b, 0xfd, 0x54, 0xb9, 0xd1, 0x5c, 0x6d, 0x36, 0xe4, 0x6a, - 0xa3, 0x59, 0xdf, 0x5c, 0x6d, 0x56, 0xd7, 0x8a, 0xc9, 0x11, 0xa5, 0xc8, 0x43, 0x24, 0xa9, 0x5a, - 0x69, 0x16, 0x53, 0xa2, 0x03, 0x97, 0x24, 0xac, 0x9a, 0x9d, 0x6e, 0xcf, 0xc1, 0xa4, 0x96, 0x76, - 0x9c, 0x33, 0xe5, 0x0a, 0x4c, 0x6b, 0xd6, 0x89, 0x6c, 0xf5, 0x0c, 0x3e, 0x4f, 0xa6, 0x34, 0xeb, - 0x44, 0xea, 0x19, 0xe2, 0x3f, 0x17, 0xe0, 0x72, 0xff, 0x63, 0xe3, 0x1c, 0x84, 0x2f, 0x20, 0xab, - 0x68, 0x1a, 0xd6, 0x64, 0x0d, 0xb7, 0x1d, 0x85, 0x3b, 0x23, 0xf7, 0x03, 0x92, 0xf8, 0x66, 0xdb, - 0xb2, 0xb7, 0xd9, 0xb6, 0xf9, 0xb2, 0x52, 0xa1, 0x15, 0x59, 0x23, 0x1c, 0xae, 0xf9, 0xa1, 0x42, - 0x28, 0x45, 0xfc, 0x9f, 0x29, 0xc8, 0x55, 0x0d, 0xad, 0x79, 0x1c, 0xeb, 0x5a, 0x72, 0x19, 0xa6, - 0x54, 0xb3, 0xd3, 0xd1, 0x1d, 0x57, 0x41, 0xec, 0x0a, 0xfd, 0x6c, 0xc0, 0x89, 0x4c, 0x8e, 0xe1, - 0x4a, 0xf9, 0xee, 0x23, 0xfa, 0x0e, 0x5c, 0x21, 0x56, 0xd3, 0x32, 0x94, 0xb6, 0xcc, 0xa4, 0xc9, - 0x8e, 0xa5, 0xb7, 0x5a, 0xd8, 0xe2, 0x1b, 0x7c, 0x77, 0x23, 0xea, 0x59, 0xe7, 0x1c, 0x15, 0xca, - 0xd0, 0x64, 0xe5, 0xa5, 0x4b, 0x7a, 0x14, 0x19, 0x7d, 0x02, 0x40, 0x96, 0x22, 0xba, 0x69, 0x68, - 0x73, 0x7b, 0x34, 0x6c, 0xd7, 0xd0, 0x35, 0x41, 0x84, 0x81, 0x5c, 0xdb, 0x68, 0x85, 0x20, 0x86, - 0xd7, 0x3d, 0xdd, 0xc2, 0xf2, 0x83, 0xae, 0x4a, 0xa1, 0x7c, 0xba, 0x9c, 0x3f, 0x3b, 0x5d, 0x04, - 0x89, 0x91, 0x1f, 0x6c, 0x57, 0x08, 0x82, 0x60, 0xbf, 0xbb, 0x2a, 0x7a, 0x05, 0xf7, 0x02, 0x3b, - 0x12, 0x64, 0xe5, 0xe5, 0xcd, 0x52, 0x1c, 0x79, 0x5f, 0x6f, 0xed, 0x63, 0x4b, 0xf6, 0x36, 0x8e, - 0xe9, 0x0e, 0x5e, 0x5a, 0xba, 0xe9, 0x33, 0x54, 0x14, 0x83, 0xd5, 0x7e, 0xd5, 0xa9, 0xd1, 0xc2, - 0x9e, 0xce, 0x88, 0xf2, 0xbb, 0xa6, 0x6e, 0x9b, 0x46, 0x29, 0xc3, 0x94, 0xcf, 0xae, 0xd0, 0x3d, - 0x28, 0x3a, 0xc7, 0x86, 0xbc, 0x8f, 0x15, 0xcb, 0xd9, 0xc5, 0x8a, 0x43, 0x56, 0x69, 0xa0, 0x25, - 0x0a, 0xce, 0xb1, 0x51, 0x0b, 0x90, 0xd1, 0x0b, 0x28, 0xea, 0x86, 0xbc, 0xd7, 0xd6, 0x5b, 0xfb, - 0x8e, 0x7c, 0x64, 0xe9, 0x0e, 0xb6, 0x4b, 0xb3, 0x54, 0x21, 0x51, 0xe3, 0xb6, 0xc1, 0x77, 0x72, - 0xb5, 0x57, 0xa4, 0x24, 0x57, 0x4d, 0x5e, 0x37, 0x9e, 0x50, 0x7e, 0x4a, 0xb4, 0xd7, 0x53, 0xe9, - 0xe9, 0x62, 0x5a, 0xfc, 0xaf, 0x02, 0xe4, 0xdd, 0xe1, 0x16, 0xe7, 0xcc, 0xb8, 0x0b, 0x45, 0xd3, - 0xc0, 0x72, 0x77, 0x5f, 0xb1, 0x31, 0xd7, 0x23, 0x5f, 0x70, 0xf2, 0xa6, 0x81, 0xb7, 0x09, 0x99, - 0xa9, 0x0b, 0x6d, 0xc3, 0xac, 0xed, 0x28, 0x2d, 0xdd, 0x68, 0x05, 0xd4, 0x3b, 0x39, 0xbe, 0x5b, - 0x5f, 0xe4, 0xdc, 0x1e, 0x3d, 0xe4, 0xa5, 0xfc, 0xa9, 0x00, 0xb3, 0xab, 0x5a, 0x47, 0x37, 0x1a, - 0xdd, 0xb6, 0x1e, 0xeb, 0x6e, 0xc1, 0x4d, 0xc8, 0xd8, 0x44, 0xa6, 0x6f, 0xf0, 0x7d, 0xec, 0x97, - 0xa6, 0x77, 0x88, 0xe5, 0x7f, 0x06, 0x05, 0x7c, 0xdc, 0xd5, 0xd9, 0xcb, 0x00, 0x06, 0x59, 0x52, - 0xe3, 0xb7, 0x2d, 0xef, 0xf3, 0x92, 0x5b, 0xbc, 0x4d, 0x9f, 0x01, 0x0a, 0x36, 0x29, 0x4e, 0xec, - 0xf2, 0x19, 0xcc, 0x51, 0xd1, 0x3b, 0x86, 0x1d, 0xb3, 0xbe, 0xc4, 0x5f, 0x84, 0xf9, 0xb0, 0xe8, - 0x38, 0xeb, 0xfd, 0x8a, 0xf7, 0xf2, 0x26, 0xb6, 0x62, 0x85, 0x9b, 0x9e, 0xae, 0xb9, 0xe0, 0x38, - 0xeb, 0xfc, 0xab, 0x02, 0x5c, 0xa5, 0xb2, 0xe9, 0xfb, 0x92, 0x3d, 0x6c, 0x3d, 0xc3, 0x8a, 0x1d, - 0x2b, 0x56, 0xbe, 0x01, 0x53, 0x0c, 0xf3, 0xd2, 0xf1, 0x39, 0x59, 0xce, 0x12, 0xcf, 0xa5, 0xe1, - 0x98, 0x16, 0xf1, 0x5c, 0xf8, 0x2d, 0x51, 0x81, 0x85, 0xa8, 0x5a, 0xc4, 0xbc, 0x1d, 0x30, 0xcb, - 0x9d, 0x46, 0x32, 0x94, 0x2b, 0xfb, 0xc4, 0x67, 0x42, 0x55, 0xc8, 0xaa, 0xf4, 0x97, 0xec, 0x9c, - 0x74, 0x31, 0x95, 0x9f, 0x1f, 0xe5, 0x6f, 0x32, 0xb6, 0xe6, 0x49, 0x17, 0x13, 0xa7, 0xd5, 0xfd, - 0x4d, 0x14, 0x15, 0x68, 0xe4, 0x48, 0x8f, 0x95, 0xce, 0x23, 0x5a, 0xd6, 0x75, 0xfd, 0xb8, 0x0e, - 0xfe, 0x45, 0x92, 0x2b, 0x81, 0x3d, 0x83, 0x17, 0x8f, 0xd5, 0x47, 0xf9, 0x3c, 0xf4, 0xba, 0x2a, - 0xd8, 0xf0, 0xc4, 0x05, 0x1a, 0x1e, 0xd8, 0x33, 0xf7, 0xa9, 0xe8, 0x33, 0x08, 0xec, 0x8a, 0xcb, - 0xac, 0x4d, 0x2e, 0xfa, 0xb9, 0x88, 0x3a, 0x66, 0x7d, 0x29, 0x8c, 0x6e, 0xa3, 0x0a, 0xa4, 0xf1, - 0x71, 0x57, 0xd6, 0xb0, 0xad, 0x72, 0xc3, 0x25, 0x0e, 0x7b, 0xaf, 0x36, 0x80, 0x07, 0xa6, 0xf1, - 0x71, 0x97, 0x10, 0xd1, 0x0e, 0x59, 0xbd, 0x5c, 0x57, 0x81, 0x56, 0xdb, 0x3e, 0x1f, 0x5e, 0xf8, - 0x23, 0x85, 0x8b, 0x2b, 0x78, 0x5e, 0x02, 0x13, 0x21, 0xfe, 0x40, 0x80, 0xb7, 0x22, 0x7b, 0x2d, - 0xce, 0x85, 0xec, 0x13, 0x48, 0xd1, 0xc6, 0x27, 0x2e, 0xd8, 0x78, 0xca, 0x25, 0xfe, 0x9e, 0x3b, - 0xc7, 0x25, 0xdc, 0x36, 0x89, 0x62, 0xbf, 0x82, 0xfd, 0xb0, 0x69, 0xb7, 0xc3, 0x13, 0x17, 0xee, - 0x70, 0x97, 0xd5, 0x33, 0x02, 0x7d, 0xd5, 0x8c, 0xd3, 0x08, 0xfc, 0xba, 0x00, 0x73, 0x9e, 0x4f, - 0x13, 0xb3, 0x7b, 0xfb, 0x01, 0x24, 0x0d, 0xf3, 0xe8, 0x22, 0x9b, 0x81, 0xa4, 0x3c, 0x59, 0x92, - 0xc2, 0x35, 0x8a, 0xb3, 0xbd, 0xff, 0x2e, 0x01, 0x99, 0xa7, 0x95, 0x38, 0x5b, 0xf9, 0x09, 0xdf, - 0x68, 0x66, 0x13, 0x3b, 0x6a, 0x28, 0x7a, 0xcf, 0x5b, 0x7e, 0x5a, 0xd9, 0xc0, 0x27, 0xee, 0x50, - 0x24, 0x5c, 0x68, 0x15, 0x32, 0xce, 0xbe, 0x85, 0xed, 0x7d, 0xb3, 0xad, 0x5d, 0xc4, 0x07, 0xf1, - 0xb9, 0x16, 0x30, 0x4c, 0x52, 0xb9, 0x6e, 0xf0, 0x82, 0x10, 0x11, 0xbc, 0x40, 0x1e, 0xe3, 0xb9, - 0x71, 0x89, 0x8b, 0x3c, 0x26, 0xe0, 0xbf, 0x4d, 0x16, 0xa7, 0xc4, 0x17, 0x00, 0xa4, 0x39, 0x71, - 0x76, 0xc9, 0xdf, 0x48, 0x42, 0x7e, 0xbb, 0x67, 0xef, 0xc7, 0x3c, 0xfa, 0x2a, 0x00, 0xdd, 0x9e, - 0x4d, 0xf1, 0xc1, 0xb1, 0xc1, 0xdb, 0x7c, 0x4e, 0x5c, 0x84, 0xdb, 0x68, 0xc6, 0xd7, 0x3c, 0x36, - 0x50, 0x8d, 0x0b, 0xc1, 0xb2, 0x1f, 0x5c, 0x71, 0x63, 0x14, 0x92, 0x6c, 0x1e, 0x1b, 0x9b, 0xd8, - 0x83, 0x90, 0x4c, 0x12, 0x26, 0x92, 0x3e, 0x81, 0x69, 0x72, 0x21, 0x3b, 0xe6, 0x45, 0xba, 0x79, - 0x8a, 0xf0, 0x34, 0x4d, 0xf4, 0x18, 0x32, 0x8c, 0x9b, 0x2c, 0x4d, 0x53, 0x74, 0x69, 0x8a, 0x6a, - 0x0b, 0x57, 0x23, 0x5d, 0x94, 0xd2, 0x94, 0x95, 0x2c, 0x44, 0xf3, 0x30, 0xb9, 0x67, 0x5a, 0x2a, - 0xa6, 0x11, 0x13, 0x69, 0x89, 0x5d, 0xb0, 0xfe, 0x5c, 0x4f, 0xa5, 0xd3, 0xc5, 0xcc, 0x7a, 0x2a, - 0x9d, 0x29, 0x82, 0xf8, 0x5b, 0x02, 0x14, 0xbc, 0x8e, 0x88, 0xd3, 0x5a, 0x57, 0x42, 0x5a, 0xbc, - 0x78, 0x57, 0x10, 0x05, 0x8a, 0xff, 0x81, 0xba, 0x2b, 0xaa, 0x79, 0x48, 0x7b, 0x26, 0xce, 0x91, - 0xf2, 0x98, 0x85, 0xce, 0x24, 0x2e, 0xda, 0xbb, 0x34, 0x8a, 0xe6, 0x01, 0xcc, 0xeb, 0x1d, 0x62, - 0xc7, 0x75, 0xa7, 0x7d, 0xc2, 0x31, 0x95, 0x83, 0xdd, 0xf7, 0xb3, 0x73, 0xfe, 0xbd, 0x8a, 0x7b, - 0x4b, 0xfc, 0x87, 0x74, 0x77, 0xda, 0x6f, 0x49, 0x9c, 0xaa, 0xae, 0x43, 0xce, 0x62, 0xa2, 0x89, - 0xcf, 0x71, 0x41, 0x6d, 0xcf, 0x78, 0xac, 0x44, 0xe1, 0xbf, 0x93, 0x80, 0xc2, 0x8b, 0x1e, 0xb6, - 0x4e, 0xbe, 0x4e, 0xea, 0xbe, 0x0d, 0x85, 0x23, 0x45, 0x77, 0xe4, 0x3d, 0xd3, 0x92, 0x7b, 0x5d, - 0x4d, 0x71, 0xdc, 0xf8, 0x8d, 0x1c, 0x21, 0x3f, 0x31, 0xad, 0x1d, 0x4a, 0x44, 0x18, 0xd0, 0x81, - 0x61, 0x1e, 0x19, 0x32, 0x21, 0x53, 0x14, 0x7b, 0x6c, 0xf0, 0x2d, 0xe3, 0xf2, 0x87, 0xff, 0xe5, - 0x74, 0xf1, 0xd1, 0x58, 0x51, 0x59, 0x34, 0x02, 0xad, 0xd7, 0xd3, 0xb5, 0xe5, 0x9d, 0x9d, 0xfa, - 0x9a, 0x54, 0xa4, 0x22, 0x5f, 0x31, 0x89, 0xcd, 0x63, 0xc3, 0x16, 0xff, 0x51, 0x02, 0x8a, 0xbe, - 0x8e, 0xe2, 0xec, 0xc8, 0x2a, 0x64, 0x5f, 0xf7, 0xb0, 0xa5, 0xbf, 0x41, 0x37, 0x02, 0x67, 0x24, - 0x66, 0xe7, 0x73, 0x98, 0x09, 0x69, 0x20, 0xf9, 0x93, 0x69, 0x20, 0x7b, 0xe4, 0x37, 0x1e, 0xdd, - 0x87, 0x59, 0xe7, 0xd8, 0x90, 0x59, 0x3c, 0x1e, 0x8b, 0xe1, 0x70, 0xc3, 0x0e, 0x0a, 0x0e, 0xd1, - 0x07, 0xa1, 0xd3, 0xf8, 0x0d, 0x5b, 0xfc, 0x23, 0x01, 0x10, 0x55, 0x54, 0x9d, 0xed, 0xe9, 0x7f, - 0x5d, 0xc6, 0xd3, 0x5d, 0x28, 0xd2, 0x08, 0x47, 0x59, 0xdf, 0x93, 0x3b, 0xba, 0x6d, 0xeb, 0x46, - 0x8b, 0x0f, 0xa8, 0x3c, 0xa5, 0xd7, 0xf7, 0x36, 0x19, 0x55, 0xfc, 0xab, 0x30, 0x17, 0x6a, 0x40, - 0x9c, 0x9d, 0x7d, 0x1d, 0x66, 0xf6, 0xcc, 0x9e, 0xa1, 0xc9, 0xec, 0x8d, 0x07, 0xdf, 0x0e, 0xcc, - 0x52, 0x1a, 0x7b, 0x9e, 0xf8, 0x3f, 0x12, 0x30, 0x2f, 0x61, 0xdb, 0x6c, 0x1f, 0xe2, 0xf8, 0x55, - 0x58, 0x03, 0xfe, 0xae, 0x45, 0x7e, 0x23, 0x4d, 0x66, 0x18, 0x33, 0x5b, 0xe6, 0xc2, 0x7b, 0xea, - 0x37, 0x47, 0x8f, 0xd8, 0xc1, 0x5d, 0x74, 0xbe, 0x27, 0x97, 0x0a, 0xed, 0xc9, 0x99, 0x50, 0xd0, - 0x5b, 0x86, 0x49, 0x6c, 0x9a, 0x8d, 0x5f, 0x1b, 0xbd, 0x8e, 0x8b, 0x54, 0x96, 0x47, 0x55, 0xb2, - 0xce, 0x58, 0x1a, 0xf8, 0xf5, 0x56, 0xaf, 0x43, 0x7d, 0xe7, 0xf2, 0x65, 0x52, 0xdf, 0xb3, 0xd3, - 0xc5, 0x7c, 0xe8, 0x9e, 0x2d, 0xe5, 0x75, 0xef, 0x9a, 0x48, 0x17, 0xbf, 0x0d, 0x97, 0xfa, 0x94, - 0x1d, 0xa7, 0xc7, 0xf3, 0x6f, 0x92, 0x70, 0x35, 0x2c, 0x3e, 0x6e, 0xfc, 0xf1, 0x75, 0xef, 0xd0, - 0x1a, 0xe4, 0x3a, 0xba, 0xf1, 0x66, 0x5b, 0x8b, 0x33, 0x1d, 0xdd, 0xf0, 0xb7, 0x71, 0x23, 0x86, - 0xc6, 0xd4, 0x57, 0x3a, 0x34, 0x14, 0x58, 0x88, 0xea, 0xbb, 0x38, 0xc7, 0xc7, 0xaf, 0x09, 0x30, - 0x13, 0xf7, 0x9e, 0xd9, 0x9b, 0xc5, 0x9b, 0x89, 0x4d, 0xc8, 0x7d, 0x05, 0x9b, 0x6c, 0xbf, 0x23, - 0x00, 0x6a, 0x5a, 0x3d, 0x83, 0x80, 0xda, 0x67, 0x66, 0x2b, 0xce, 0x66, 0xce, 0xc3, 0xa4, 0x6e, - 0x68, 0xf8, 0x98, 0x36, 0x33, 0x25, 0xb1, 0x8b, 0xd0, 0xab, 0xc3, 0xe4, 0x58, 0xaf, 0x0e, 0xc5, - 0xcf, 0x61, 0x2e, 0x54, 0xc5, 0x38, 0xdb, 0xff, 0x8f, 0x13, 0x30, 0xc7, 0x1b, 0x12, 0xfb, 0xf6, - 0xe2, 0x37, 0x61, 0xb2, 0x4d, 0x64, 0x8e, 0xe8, 0x67, 0xfa, 0x4c, 0xb7, 0x9f, 0x69, 0x61, 0xf4, - 0x73, 0x00, 0x5d, 0x0b, 0x1f, 0xca, 0x8c, 0x35, 0x39, 0x16, 0x6b, 0x86, 0x70, 0x50, 0x02, 0xfa, - 0x05, 0x28, 0x90, 0xf9, 0xdc, 0xb5, 0xcc, 0xae, 0x69, 0x13, 0x97, 0xc5, 0x1e, 0x0f, 0xe5, 0xcc, - 0x9e, 0x9d, 0x2e, 0xe6, 0x36, 0x75, 0x63, 0x9b, 0x33, 0x36, 0x1b, 0x12, 0x31, 0x0c, 0xde, 0xa5, - 0x2d, 0xfe, 0x27, 0x01, 0xe6, 0xbf, 0xb2, 0xad, 0xd8, 0xff, 0x1f, 0xba, 0x12, 0x5f, 0x42, 0x91, - 0xfe, 0xa8, 0x1b, 0x7b, 0x66, 0x9c, 0x9b, 0xe2, 0xdf, 0x13, 0x60, 0x36, 0x20, 0x38, 0x4e, 0xff, - 0xe4, 0x8d, 0xf4, 0x24, 0xfe, 0x22, 0xf1, 0x58, 0x82, 0x83, 0x3c, 0xce, 0x29, 0xf4, 0x07, 0x09, - 0xb8, 0x5c, 0x61, 0xaf, 0x90, 0xdd, 0x98, 0x8a, 0x38, 0x47, 0x46, 0x09, 0xa6, 0x0f, 0xb1, 0x65, - 0xeb, 0x26, 0x5b, 0x3d, 0x73, 0x92, 0x7b, 0x89, 0x16, 0x20, 0x6d, 0x1b, 0x4a, 0xd7, 0xde, 0x37, - 0xdd, 0x77, 0x67, 0xde, 0xb5, 0x17, 0xff, 0x31, 0xf9, 0xe6, 0xf1, 0x1f, 0x53, 0xa3, 0xe3, 0x3f, - 0xa6, 0x7f, 0x82, 0xf8, 0x0f, 0xfe, 0xa2, 0xea, 0x3f, 0x0a, 0x70, 0x65, 0x40, 0x73, 0x71, 0x8e, - 0x96, 0xef, 0x42, 0x56, 0xe5, 0x82, 0x89, 0xbd, 0x65, 0x6f, 0xe1, 0xea, 0xa4, 0xd8, 0x1b, 0xc2, - 0x8e, 0xb3, 0xd3, 0x45, 0x70, 0xab, 0x5a, 0x5f, 0xe3, 0xca, 0x21, 0xbf, 0x35, 0xf1, 0x57, 0x72, - 0x50, 0xa8, 0x1e, 0xb3, 0x1d, 0xe8, 0x06, 0x5b, 0xe5, 0xd1, 0x13, 0x48, 0x77, 0x2d, 0xf3, 0x50, - 0x77, 0x9b, 0x91, 0x0f, 0xbd, 0xfc, 0x77, 0x9b, 0xd1, 0xc7, 0xb5, 0xcd, 0x39, 0x24, 0x8f, 0x17, - 0x35, 0x21, 0xf3, 0xcc, 0x54, 0x95, 0xf6, 0x13, 0xbd, 0xed, 0x8e, 0xfc, 0xf7, 0xcf, 0x17, 0xb4, - 0xec, 0xf1, 0x6c, 0x2b, 0xce, 0xbe, 0xdb, 0x09, 0x1e, 0x11, 0xd5, 0x21, 0x5d, 0x73, 0x9c, 0x2e, - 0xb9, 0xc9, 0x6d, 0xc7, 0x9d, 0x31, 0x84, 0x12, 0x16, 0x37, 0x56, 0xd4, 0x65, 0x47, 0x4d, 0x98, - 0x7d, 0x4a, 0x4f, 0x38, 0x55, 0xda, 0x66, 0x4f, 0xab, 0x98, 0xc6, 0x9e, 0xde, 0xe2, 0x76, 0xf7, - 0xf6, 0x18, 0x32, 0x9f, 0x56, 0x1a, 0xd2, 0xa0, 0x00, 0xb4, 0x0a, 0xe9, 0xc6, 0x23, 0x2e, 0x8c, - 0xb9, 0x65, 0xb7, 0xc6, 0x10, 0xd6, 0x78, 0x24, 0x79, 0x6c, 0x68, 0x1d, 0xb2, 0xab, 0x5f, 0xf4, - 0x2c, 0xcc, 0xa5, 0x4c, 0x0d, 0x8d, 0x3c, 0xe8, 0x97, 0x42, 0xb9, 0xa4, 0x20, 0x33, 0x6a, 0x40, - 0xfe, 0x95, 0x69, 0x1d, 0xb4, 0x4d, 0xc5, 0x6d, 0xe1, 0x34, 0x15, 0xf7, 0x8d, 0x31, 0xc4, 0xb9, - 0x8c, 0x52, 0x9f, 0x08, 0xf4, 0x6d, 0x28, 0x90, 0xce, 0x68, 0x2a, 0xbb, 0x6d, 0xb7, 0x92, 0x69, - 0x2a, 0xf5, 0xdd, 0x31, 0xa4, 0x7a, 0x9c, 0xee, 0x2b, 0x90, 0x3e, 0x51, 0x0b, 0x12, 0xe4, 0x42, - 0x83, 0x00, 0x21, 0x48, 0x75, 0x49, 0x7f, 0x0b, 0x34, 0x36, 0x88, 0xfe, 0x46, 0xef, 0xc1, 0xb4, - 0x61, 0x6a, 0xd8, 0x9d, 0x21, 0xb9, 0xf2, 0xfc, 0xd9, 0xe9, 0xe2, 0xd4, 0x96, 0xa9, 0x31, 0x87, - 0x84, 0xff, 0x92, 0xa6, 0x48, 0xa1, 0xba, 0xb6, 0xb0, 0x04, 0x29, 0xd2, 0xef, 0xc4, 0x30, 0xed, - 0x2a, 0x36, 0xde, 0xb1, 0x74, 0x2e, 0xcd, 0xbd, 0x5c, 0xf8, 0x67, 0x09, 0x48, 0x34, 0x1e, 0x11, - 0x97, 0x7b, 0xb7, 0xa7, 0x1e, 0x60, 0x87, 0xdf, 0xe7, 0x57, 0xd4, 0x15, 0xb7, 0xf0, 0x9e, 0xce, - 0x3c, 0xa3, 0x8c, 0xc4, 0xaf, 0xd0, 0x3b, 0x00, 0x8a, 0xaa, 0x62, 0xdb, 0x96, 0xdd, 0x93, 0x6f, - 0x19, 0x29, 0xc3, 0x28, 0x1b, 0xf8, 0x84, 0xb0, 0xd9, 0x58, 0xb5, 0xb0, 0xe3, 0x06, 0x36, 0xb1, - 0x2b, 0xc2, 0xe6, 0xe0, 0x4e, 0x57, 0x76, 0xcc, 0x03, 0x6c, 0xd0, 0x71, 0x92, 0x21, 0xa6, 0xa6, - 0xd3, 0x6d, 0x12, 0x02, 0xb1, 0x92, 0xd8, 0xd0, 0x7c, 0x93, 0x96, 0x91, 0xbc, 0x6b, 0x22, 0xd2, - 0xc2, 0x2d, 0x9d, 0x1f, 0xe1, 0xca, 0x48, 0xfc, 0x8a, 0x68, 0x49, 0xe9, 0x39, 0xfb, 0xb4, 0x27, - 0x32, 0x12, 0xfd, 0x8d, 0x6e, 0x43, 0x81, 0xc5, 0x42, 0xca, 0xd8, 0x50, 0x65, 0x6a, 0x5c, 0x33, - 0xf4, 0x76, 0x8e, 0x91, 0xab, 0x86, 0x4a, 0x4c, 0x29, 0x7a, 0x04, 0x9c, 0x20, 0x1f, 0x74, 0x6c, - 0xa2, 0x53, 0x20, 0xa5, 0xca, 0x85, 0xb3, 0xd3, 0xc5, 0x6c, 0x83, 0xde, 0xd8, 0xd8, 0x6c, 0xd4, - 0xd7, 0xa4, 0x2c, 0x2b, 0xb5, 0xd1, 0xb1, 0xeb, 0xda, 0xc2, 0x6f, 0x08, 0x90, 0x7c, 0x5a, 0x69, - 0x5c, 0x58, 0x65, 0x6e, 0x45, 0x93, 0x81, 0x8a, 0xde, 0x81, 0xc2, 0xae, 0xde, 0x6e, 0xeb, 0x46, - 0x8b, 0x78, 0x41, 0xdf, 0xc5, 0xaa, 0xab, 0xb0, 0x3c, 0x27, 0x6f, 0x33, 0x2a, 0x5a, 0x82, 0xac, - 0x6a, 0x61, 0x0d, 0x1b, 0x8e, 0xae, 0xb4, 0x6d, 0xae, 0xb9, 0x20, 0x69, 0xe1, 0x97, 0x05, 0x98, - 0xa4, 0x33, 0x00, 0xbd, 0x0d, 0x19, 0xd5, 0x34, 0x1c, 0x45, 0x37, 0xb8, 0x29, 0xcb, 0x48, 0x3e, - 0x61, 0x68, 0xf5, 0xae, 0xc3, 0x8c, 0xa2, 0xaa, 0x66, 0xcf, 0x70, 0x64, 0x43, 0xe9, 0x60, 0x5e, - 0xcd, 0x2c, 0xa7, 0x6d, 0x29, 0x1d, 0x8c, 0x16, 0xc1, 0xbd, 0xf4, 0x0e, 0x34, 0x66, 0x24, 0xe0, - 0xa4, 0x0d, 0x7c, 0xb2, 0xf0, 0xc7, 0x02, 0xa4, 0xdd, 0x39, 0x43, 0xaa, 0xd1, 0xc2, 0x06, 0xb6, - 0x14, 0xc7, 0xf4, 0xaa, 0xe1, 0x11, 0xfa, 0x97, 0xca, 0x8c, 0xbf, 0x54, 0xce, 0xc3, 0xa4, 0x43, - 0xa6, 0x05, 0xaf, 0x01, 0xbb, 0xa0, 0xdb, 0xd1, 0x6d, 0xa5, 0xc5, 0x76, 0xe3, 0x32, 0x12, 0xbb, - 0x20, 0x8d, 0xe1, 0x21, 0xb5, 0x4c, 0x23, 0xfc, 0x8a, 0xd4, 0x94, 0x05, 0x7e, 0xee, 0xe2, 0x96, - 0x6e, 0xd0, 0xb1, 0x94, 0x94, 0x80, 0x92, 0xca, 0x84, 0x82, 0xde, 0x82, 0x0c, 0x2b, 0x80, 0x0d, - 0x8d, 0x0e, 0xa8, 0xa4, 0x94, 0xa6, 0x84, 0xaa, 0xa1, 0x2d, 0x60, 0xc8, 0x78, 0x93, 0x93, 0x74, - 0x5b, 0xcf, 0xf6, 0x14, 0x49, 0x7f, 0xa3, 0xf7, 0x61, 0xfe, 0x75, 0x4f, 0x69, 0xeb, 0x7b, 0x74, - 0xa3, 0x8d, 0x14, 0x63, 0x3a, 0x63, 0x2d, 0x41, 0xde, 0x3d, 0x2a, 0x81, 0xaa, 0xce, 0x9d, 0xcb, - 0x49, 0x7f, 0x2e, 0x8b, 0xbf, 0x2f, 0xc0, 0x2c, 0x8b, 0xea, 0x61, 0xc1, 0xa8, 0xf1, 0xf9, 0x21, - 0x1f, 0x43, 0x46, 0x53, 0x1c, 0x85, 0x1d, 0xd1, 0x4c, 0x8c, 0x3c, 0xa2, 0xe9, 0x1d, 0x25, 0x50, - 0x1c, 0x85, 0x1e, 0xd3, 0x44, 0x90, 0x22, 0xbf, 0xd9, 0x69, 0x56, 0x89, 0xfe, 0x16, 0x3f, 0x03, - 0x14, 0xac, 0x68, 0x9c, 0x1e, 0xd9, 0x3d, 0xb8, 0x44, 0x74, 0x5d, 0x35, 0x54, 0xeb, 0xa4, 0xeb, - 0xe8, 0xa6, 0xf1, 0x9c, 0xfe, 0xb5, 0x51, 0x31, 0xf0, 0x5e, 0x8a, 0xbe, 0x8e, 0x12, 0xff, 0x70, - 0x0a, 0x72, 0xd5, 0xe3, 0xae, 0x69, 0xc5, 0xba, 0x8b, 0x55, 0x86, 0x69, 0x0e, 0xf4, 0x47, 0xbc, - 0x17, 0xee, 0x33, 0xe6, 0xee, 0x2b, 0x57, 0xce, 0x88, 0xca, 0x00, 0x2c, 0x60, 0x94, 0x06, 0x05, - 0x25, 0x2f, 0xf0, 0xa6, 0x8c, 0xb2, 0x11, 0x2a, 0xda, 0x82, 0x6c, 0xe7, 0x50, 0x55, 0xe5, 0x3d, - 0xbd, 0xed, 0xf0, 0xb8, 0xbb, 0xe8, 0x10, 0xf1, 0xcd, 0x97, 0x95, 0xca, 0x13, 0x5a, 0x88, 0x85, - 0xc0, 0xf9, 0xd7, 0x12, 0x10, 0x09, 0xec, 0x37, 0x7a, 0x17, 0xf8, 0x91, 0x1a, 0xd9, 0x76, 0x4f, - 0xc9, 0x95, 0x73, 0x67, 0xa7, 0x8b, 0x19, 0x89, 0x52, 0x1b, 0x8d, 0xa6, 0x94, 0x61, 0x05, 0x1a, - 0xb6, 0x83, 0x6e, 0x40, 0xce, 0xec, 0xe8, 0x8e, 0xec, 0x3a, 0x49, 0xdc, 0xa3, 0x9c, 0x21, 0x44, - 0xd7, 0x89, 0x42, 0x4d, 0xb8, 0x83, 0x0d, 0x3a, 0xda, 0x49, 0x3b, 0xe5, 0x5d, 0xb6, 0xf9, 0xe8, - 0xb0, 0x19, 0x2d, 0x9b, 0x5d, 0x47, 0xef, 0xe8, 0x5f, 0xd0, 0x37, 0xd3, 0xfc, 0xa5, 0xd1, 0x0d, - 0x56, 0x9c, 0xb4, 0xaf, 0x4c, 0x77, 0x25, 0x79, 0xd9, 0xe7, 0x81, 0xa2, 0xe8, 0x6f, 0x0a, 0x70, - 0x99, 0x2b, 0x52, 0xde, 0xa5, 0x31, 0xee, 0x4a, 0x5b, 0x77, 0x4e, 0xe4, 0x83, 0xc3, 0x52, 0x9a, - 0xfa, 0xad, 0x3f, 0x1b, 0xd9, 0x21, 0x81, 0x71, 0xb0, 0xec, 0x76, 0xcb, 0xc9, 0x33, 0xce, 0xbc, - 0x71, 0x58, 0x35, 0x1c, 0xeb, 0xa4, 0x7c, 0xe5, 0xec, 0x74, 0x71, 0x6e, 0xf0, 0xee, 0x4b, 0x69, - 0xce, 0x1e, 0x64, 0x41, 0x35, 0x00, 0xec, 0x8d, 0x43, 0xba, 0x62, 0x44, 0xfb, 0x1f, 0x91, 0x03, - 0x56, 0x0a, 0xf0, 0xa2, 0xbb, 0x50, 0xe4, 0x27, 0x5b, 0xf6, 0xf4, 0x36, 0x96, 0x6d, 0xfd, 0x0b, - 0x4c, 0xd7, 0x96, 0xa4, 0x94, 0x67, 0x74, 0x22, 0xa2, 0xa1, 0x7f, 0x81, 0x17, 0xbe, 0x0b, 0xa5, - 0x61, 0xb5, 0x0f, 0x4e, 0x81, 0x0c, 0x7b, 0x23, 0xfb, 0x51, 0x78, 0x3b, 0x66, 0x8c, 0xa1, 0xca, - 0xb7, 0x64, 0x3e, 0x4e, 0x7c, 0x24, 0x88, 0xff, 0x24, 0x01, 0xb9, 0x72, 0xaf, 0x7d, 0xf0, 0xbc, - 0xdb, 0xe8, 0x75, 0x3a, 0x8a, 0x75, 0x42, 0xcc, 0x20, 0x33, 0x14, 0xa4, 0x82, 0x02, 0x33, 0x83, - 0xd4, 0x12, 0xe8, 0x5f, 0x60, 0xb2, 0x38, 0x05, 0x4f, 0x65, 0xb3, 0x18, 0x7e, 0xda, 0x86, 0xc0, - 0x51, 0x6b, 0xf3, 0xc8, 0x46, 0x1f, 0x41, 0x29, 0x50, 0x90, 0xee, 0x9d, 0xc8, 0xd8, 0x70, 0x2c, - 0x1d, 0xb3, 0xfd, 0xbf, 0xa4, 0x14, 0x88, 0x97, 0xa9, 0x93, 0xdb, 0x55, 0x76, 0x17, 0x35, 0x61, - 0x86, 0x14, 0x3c, 0x91, 0xe9, 0x12, 0xe2, 0xee, 0xcf, 0x3e, 0x88, 0x68, 0x56, 0xa8, 0xde, 0xcb, - 0x54, 0x3f, 0x15, 0xca, 0x43, 0x7f, 0x4a, 0x59, 0xec, 0x53, 0x16, 0x3e, 0x85, 0x62, 0x7f, 0x81, - 0xa0, 0x2e, 0x53, 0x4c, 0x97, 0xf3, 0x41, 0x5d, 0x26, 0x03, 0x7a, 0x5a, 0x4f, 0xa5, 0x53, 0xc5, - 0x49, 0xf1, 0x2f, 0x92, 0x90, 0x77, 0x87, 0x59, 0x9c, 0x40, 0xa7, 0x0c, 0x93, 0x64, 0x50, 0xb8, - 0x31, 0x1e, 0xb7, 0x47, 0x8c, 0x6e, 0x1e, 0x35, 0x4e, 0x06, 0x8b, 0x0b, 0x92, 0x29, 0x6b, 0x1c, - 0x06, 0x67, 0xe1, 0x97, 0x13, 0x90, 0xa2, 0xd8, 0xe2, 0x01, 0xa4, 0xe8, 0x42, 0x21, 0x8c, 0xb3, - 0x50, 0xd0, 0xa2, 0xde, 0x72, 0x96, 0x08, 0xb8, 0xa6, 0xc4, 0xe7, 0xdb, 0x57, 0x3e, 0x78, 0xf0, - 0x90, 0x1a, 0x9b, 0x19, 0x89, 0x5f, 0xa1, 0x32, 0x0d, 0x3b, 0x32, 0x2d, 0x07, 0x6b, 0xdc, 0xa7, - 0x5f, 0x3a, 0xaf, 0x7f, 0xdd, 0x45, 0xc9, 0xe5, 0x43, 0x57, 0x21, 0x49, 0xac, 0xd8, 0x34, 0x0b, - 0x52, 0x38, 0x3b, 0x5d, 0x4c, 0x12, 0xfb, 0x45, 0x68, 0x68, 0x05, 0xb2, 0x61, 0x93, 0x41, 0x3c, - 0x38, 0x6a, 0x18, 0x03, 0xd3, 0x1d, 0xda, 0xde, 0xd4, 0x62, 0x78, 0x96, 0xf7, 0xf1, 0x5f, 0xa6, - 0x20, 0x57, 0xef, 0xc4, 0xbd, 0xa4, 0xac, 0x86, 0x7b, 0x38, 0x0a, 0x08, 0x85, 0x1e, 0x1a, 0xd1, - 0xc1, 0xa1, 0x15, 0x3c, 0x79, 0xb1, 0x15, 0xbc, 0x4e, 0x3c, 0x65, 0x9e, 0x66, 0x21, 0x39, 0x04, - 0xf3, 0x84, 0x9f, 0x4f, 0xfd, 0x14, 0x89, 0xf0, 0xf8, 0xe7, 0x28, 0x68, 0xa0, 0xc9, 0xa7, 0xd4, - 0x21, 0x67, 0xa3, 0x6c, 0x6a, 0xfc, 0x51, 0x36, 0x8d, 0x0d, 0x8d, 0x2e, 0x6a, 0x61, 0x8b, 0x3a, - 0xfd, 0xe6, 0x16, 0x75, 0xc1, 0xe1, 0x83, 0xf5, 0x63, 0x48, 0x6a, 0xba, 0xdb, 0x39, 0xe3, 0x2f, - 0xd5, 0x84, 0xe9, 0x9c, 0x51, 0x9b, 0x0a, 0x8e, 0x5a, 0x36, 0x4a, 0x16, 0xea, 0x00, 0xbe, 0x6e, - 0xd0, 0x12, 0x4c, 0x99, 0x6d, 0xcd, 0x3d, 0x48, 0x92, 0x2b, 0x67, 0xce, 0x4e, 0x17, 0x27, 0x9f, - 0xb7, 0xb5, 0xfa, 0x9a, 0x34, 0x69, 0xb6, 0xb5, 0xba, 0x46, 0x73, 0x5c, 0xe0, 0x23, 0xd9, 0x8b, - 0x32, 0x9b, 0x91, 0xa6, 0x0d, 0x7c, 0xb4, 0x86, 0x6d, 0x95, 0x0f, 0xb8, 0xdf, 0x16, 0x20, 0xef, - 0xea, 0x3e, 0x5e, 0xa3, 0x92, 0xd6, 0x3b, 0x7c, 0x92, 0x25, 0x2f, 0x36, 0xc9, 0x5c, 0x3e, 0x7e, - 0xbc, 0xf6, 0x57, 0x05, 0x1e, 0x37, 0xdc, 0x50, 0x15, 0x87, 0x38, 0x15, 0x31, 0x4e, 0x8c, 0x7b, - 0x50, 0xb4, 0x14, 0x43, 0x33, 0x3b, 0xfa, 0x17, 0x98, 0x6d, 0x84, 0xda, 0xfc, 0xad, 0x65, 0xc1, - 0xa3, 0xd3, 0x5d, 0x3f, 0x5b, 0xfc, 0xe3, 0x04, 0x8f, 0x31, 0xf6, 0xaa, 0x11, 0xa7, 0xba, 0xbe, - 0x03, 0xb3, 0xfd, 0x09, 0x47, 0xdc, 0xd9, 0xfa, 0x5e, 0x84, 0xbc, 0xa8, 0x8a, 0xb0, 0x58, 0x41, - 0x37, 0x70, 0xbd, 0x2f, 0xf9, 0x88, 0x8d, 0x2a, 0x90, 0x0d, 0xe6, 0x31, 0x49, 0x8e, 0x9d, 0xc7, - 0x04, 0x2c, 0x2f, 0x7b, 0xc9, 0xc2, 0xcf, 0xc3, 0x24, 0xbd, 0xfd, 0x06, 0x26, 0x9a, 0xf7, 0xe6, - 0x9f, 0x27, 0xe0, 0x26, 0xad, 0xfd, 0x4b, 0x6c, 0xe9, 0x7b, 0x27, 0xdb, 0x96, 0xe9, 0x60, 0xd5, - 0xc1, 0x9a, 0x7f, 0x0a, 0x24, 0x56, 0xbb, 0x97, 0xe9, 0xba, 0x0f, 0xb8, 0x50, 0xbc, 0x98, 0xc7, - 0x85, 0x36, 0xa0, 0xc0, 0x23, 0x03, 0x94, 0xb6, 0x7e, 0x88, 0x65, 0xc5, 0xb9, 0xc8, 0xea, 0x96, - 0x63, 0xbc, 0xab, 0x84, 0x75, 0xd5, 0x41, 0x1a, 0x64, 0xb8, 0x30, 0x5d, 0xe3, 0xe9, 0x77, 0x9e, - 0xfe, 0x64, 0x1b, 0x8a, 0x69, 0x16, 0x9e, 0x50, 0x5f, 0x93, 0xd2, 0x4c, 0x72, 0x5d, 0x13, 0xff, - 0xb3, 0x00, 0xb7, 0xce, 0x51, 0x71, 0x9c, 0x43, 0x77, 0x01, 0xd2, 0x87, 0xe4, 0x41, 0x3a, 0xd7, - 0x71, 0x5a, 0xf2, 0xae, 0xd1, 0x26, 0xe4, 0xf6, 0x14, 0xbd, 0xed, 0x0f, 0xe9, 0xe1, 0xe1, 0x85, - 0xd1, 0x91, 0xae, 0x33, 0x8c, 0x9d, 0x8d, 0x61, 0xf1, 0x37, 0x13, 0x30, 0xbb, 0xaa, 0x69, 0x8d, - 0x06, 0xb7, 0x81, 0xf1, 0x8d, 0x14, 0x17, 0x64, 0x26, 0x7c, 0x90, 0x89, 0xde, 0x03, 0xa4, 0xe9, - 0x36, 0x4b, 0xf3, 0x61, 0xef, 0x2b, 0x9a, 0x79, 0xe4, 0x87, 0x55, 0xcc, 0xba, 0x77, 0x1a, 0xee, - 0x0d, 0xd4, 0x00, 0x8a, 0x76, 0x64, 0xdb, 0x51, 0xbc, 0xf7, 0x46, 0xb7, 0xc6, 0x3a, 0xae, 0xc5, - 0x60, 0x90, 0x77, 0x29, 0x65, 0x88, 0x1c, 0xfa, 0x93, 0xf8, 0xed, 0x3a, 0x69, 0xba, 0x23, 0x2b, - 0xb6, 0x7b, 0x36, 0x87, 0x25, 0x18, 0xc9, 0x33, 0xfa, 0xaa, 0xcd, 0x8e, 0xdc, 0xb0, 0xc3, 0x04, - 0xbe, 0x6a, 0xe2, 0x84, 0xc4, 0xff, 0x40, 0x80, 0xbc, 0x84, 0xf7, 0x2c, 0x6c, 0xc7, 0xba, 0x29, - 0xf0, 0x04, 0x66, 0x2c, 0x26, 0x55, 0xde, 0xb3, 0xcc, 0xce, 0x45, 0xe6, 0x55, 0x96, 0x33, 0x3e, - 0xb1, 0xcc, 0x0e, 0x37, 0x2c, 0x2f, 0xa1, 0xe0, 0xd5, 0x31, 0xce, 0xc6, 0xff, 0x3e, 0x3d, 0x8a, - 0xcc, 0x04, 0xc7, 0x1d, 0xdf, 0x10, 0xaf, 0x06, 0xe8, 0x8b, 0xaa, 0x60, 0x45, 0xe3, 0x54, 0xc3, - 0x7f, 0x17, 0x20, 0xdf, 0xe8, 0xed, 0xb2, 0x3c, 0x52, 0xf1, 0x69, 0xa0, 0x0a, 0x99, 0x36, 0xde, - 0x73, 0xe4, 0x37, 0x0a, 0x83, 0x4f, 0x13, 0x56, 0x7a, 0x08, 0xe0, 0x29, 0x80, 0x45, 0x8f, 0xaf, - 0x51, 0x39, 0xc9, 0x0b, 0xca, 0xc9, 0x50, 0x5e, 0x42, 0x16, 0xff, 0x69, 0x02, 0x0a, 0x5e, 0x33, - 0xe3, 0xb4, 0x92, 0xaf, 0x42, 0xd6, 0x21, 0x79, 0x11, 0xeb, 0x30, 0xcb, 0x43, 0x3a, 0xa2, 0x2d, - 0xc4, 0x32, 0xcc, 0x51, 0xc7, 0x45, 0x56, 0xba, 0xdd, 0xb6, 0xee, 0xc2, 0x5d, 0x6a, 0x7f, 0x52, - 0xd2, 0x2c, 0xbd, 0xb5, 0xca, 0xee, 0x50, 0xa0, 0x4b, 0xc6, 0xdc, 0x9e, 0x85, 0xf1, 0x17, 0x58, - 0xa6, 0xc8, 0xeb, 0x22, 0x21, 0x2b, 0x59, 0xc6, 0xd8, 0x20, 0x7c, 0x7c, 0xcc, 0xbd, 0x82, 0x59, - 0xaa, 0xd3, 0xb8, 0x8f, 0xdd, 0x8a, 0x7f, 0x2f, 0x01, 0x28, 0x28, 0xf9, 0xab, 0xeb, 0x8b, 0x44, - 0x7c, 0x7d, 0xf1, 0x2e, 0x20, 0x16, 0xb4, 0x68, 0xcb, 0x5d, 0x6c, 0xc9, 0x36, 0x56, 0x4d, 0x9e, - 0xe3, 0x48, 0x90, 0x8a, 0xfc, 0xce, 0x36, 0xb6, 0x1a, 0x94, 0x8e, 0x1e, 0x03, 0xf8, 0x1e, 0x19, - 0x5f, 0x30, 0x46, 0x3a, 0x64, 0x52, 0xc6, 0x73, 0xc5, 0xc4, 0xef, 0x2d, 0xc0, 0x0c, 0xd7, 0xe1, - 0x8e, 0xa1, 0x9b, 0x06, 0x7a, 0x00, 0xc9, 0x16, 0xdf, 0xfa, 0xcf, 0x46, 0x6e, 0xd3, 0xf9, 0x49, - 0xdc, 0x6a, 0x13, 0x12, 0x29, 0x4b, 0x58, 0xba, 0x3d, 0x27, 0xc2, 0x31, 0xf2, 0x03, 0xb1, 0x83, - 0x2c, 0xdd, 0x9e, 0x83, 0x1a, 0x50, 0x50, 0xfd, 0x24, 0x55, 0x32, 0x61, 0x4f, 0x0e, 0x05, 0x51, - 0x91, 0x69, 0xc1, 0x6a, 0x13, 0x52, 0x5e, 0x0d, 0xdd, 0x40, 0x95, 0x60, 0x6e, 0xa4, 0xd4, 0x40, - 0x94, 0x97, 0x7f, 0xbe, 0x37, 0x9c, 0x97, 0xa9, 0x36, 0x11, 0x48, 0xa1, 0x84, 0x3e, 0x86, 0x29, - 0x8d, 0x66, 0xe1, 0xe1, 0x23, 0x3a, 0x6a, 0xd0, 0x85, 0x92, 0x1d, 0xd5, 0x26, 0x24, 0xce, 0x81, - 0xd6, 0x61, 0x86, 0xfd, 0x62, 0x6e, 0x0a, 0x47, 0x96, 0xb7, 0x86, 0x4b, 0x08, 0x2c, 0x04, 0xb5, - 0x09, 0x29, 0xab, 0xf9, 0x54, 0xf4, 0x4d, 0x48, 0xd9, 0xaa, 0xe2, 0x62, 0xcb, 0x6b, 0x43, 0x52, - 0x6a, 0xf8, 0xcc, 0xb4, 0x34, 0x7a, 0xcc, 0xd2, 0x32, 0x3a, 0xc7, 0xee, 0x36, 0x5f, 0x54, 0xf5, - 0x43, 0x47, 0xb7, 0x49, 0xf5, 0x31, 0x25, 0xa0, 0xa7, 0x90, 0x55, 0x88, 0xbf, 0x27, 0xd3, 0xc3, - 0x8f, 0x74, 0x5f, 0x2f, 0xfa, 0xe5, 0xfa, 0xc0, 0x61, 0xd5, 0x1a, 0x3d, 0x1f, 0xee, 0x12, 0x7d, - 0x41, 0x1d, 0x6c, 0xb5, 0x70, 0x29, 0x3b, 0x5a, 0x50, 0x30, 0xb6, 0xcb, 0x13, 0x44, 0x89, 0xc4, - 0xef, 0xf3, 0x4e, 0x1e, 0xd3, 0x46, 0xcd, 0x0c, 0x7d, 0x9d, 0x1b, 0x71, 0x6c, 0xa7, 0x36, 0x21, - 0xcd, 0xec, 0x07, 0xc8, 0x68, 0x19, 0x12, 0x2d, 0xb5, 0x94, 0x1b, 0x3a, 0x43, 0xbc, 0xa3, 0x29, - 0xb5, 0x09, 0x29, 0xd1, 0x52, 0xd1, 0xa7, 0x90, 0x66, 0xe7, 0x0c, 0x8e, 0x8d, 0x52, 0x7e, 0xa8, - 0x9d, 0x08, 0x9f, 0xd6, 0xa8, 0x4d, 0x48, 0xf4, 0x68, 0x03, 0x79, 0xde, 0x36, 0xe4, 0x2d, 0x16, - 0x1c, 0xe7, 0x86, 0xb2, 0x16, 0x87, 0xbe, 0xe2, 0x8e, 0x8a, 0x66, 0xad, 0x51, 0xcf, 0x3f, 0x40, - 0x47, 0xdf, 0x81, 0xf9, 0xb0, 0x44, 0x3e, 0xd2, 0x66, 0x87, 0xbe, 0xae, 0x1d, 0x1a, 0x59, 0x59, - 0x9b, 0x90, 0x90, 0x35, 0x70, 0x13, 0x7d, 0x08, 0x93, 0xac, 0xd7, 0x10, 0x15, 0x19, 0x15, 0xb5, - 0xd1, 0xd7, 0x61, 0xac, 0x3c, 0x19, 0xfc, 0x0e, 0x8f, 0x10, 0x93, 0xdb, 0x66, 0xab, 0x34, 0x37, - 0x74, 0xf0, 0x0f, 0xc6, 0xba, 0x91, 0xc1, 0xef, 0xf8, 0x54, 0xd2, 0xef, 0x16, 0xbb, 0xc3, 0xc3, - 0x8a, 0xe6, 0x87, 0xf6, 0x7b, 0x44, 0xe0, 0x58, 0x8d, 0xc6, 0xee, 0xfb, 0x64, 0x52, 0x35, 0x8b, - 0x65, 0x81, 0x91, 0xe9, 0x9c, 0xba, 0x34, 0xb4, 0x6a, 0x83, 0x69, 0x72, 0x6a, 0xd4, 0x47, 0xf2, - 0xa8, 0xe8, 0x25, 0x14, 0x79, 0xae, 0x06, 0xff, 0x9d, 0xc2, 0x65, 0x2a, 0xef, 0x5e, 0xa4, 0xe9, - 0x8a, 0x8a, 0xc9, 0xa9, 0x4d, 0x48, 0x05, 0x35, 0x7c, 0x07, 0x7d, 0x06, 0xb3, 0x54, 0x9e, 0xac, - 0xfa, 0xe9, 0x35, 0x4a, 0xa5, 0x81, 0x64, 0x0d, 0xc3, 0x33, 0x71, 0xb8, 0x92, 0x8b, 0x6a, 0xdf, - 0x2d, 0x32, 0x8c, 0x75, 0x43, 0x77, 0xa8, 0x95, 0x5d, 0x18, 0x3a, 0x8c, 0xc3, 0x49, 0xfd, 0xc8, - 0x30, 0xd6, 0x19, 0x85, 0x0c, 0x63, 0x87, 0xc7, 0x9c, 0xf1, 0xee, 0x78, 0x7b, 0xe8, 0x30, 0x8e, - 0x0a, 0x4e, 0x23, 0xc3, 0xd8, 0x09, 0xd2, 0xc9, 0x30, 0x66, 0x06, 0xa2, 0x4f, 0xee, 0x3b, 0x43, - 0x87, 0xf1, 0xd0, 0x43, 0xc8, 0x64, 0x18, 0x2b, 0x03, 0x37, 0xd1, 0x1a, 0x00, 0x73, 0x67, 0xe8, - 0xa2, 0x78, 0x6d, 0xe8, 0x62, 0xd0, 0x1f, 0x7b, 0x46, 0x16, 0x83, 0xb6, 0x4b, 0x23, 0x86, 0x8c, - 0x82, 0x25, 0x99, 0xbe, 0x42, 0x2d, 0x2d, 0x0e, 0x35, 0x64, 0x03, 0xaf, 0x3b, 0x89, 0x21, 0x3b, - 0xf2, 0x88, 0x64, 0x55, 0x61, 0x7b, 0xbe, 0xa5, 0xa5, 0xe1, 0x66, 0x39, 0xf8, 0xea, 0x87, 0x9a, - 0x65, 0x4a, 0x40, 0xab, 0x90, 0x21, 0x6b, 0xfe, 0x09, 0x35, 0x43, 0xd7, 0x87, 0xfa, 0xa4, 0x7d, - 0x87, 0x53, 0x6a, 0x13, 0x52, 0xfa, 0x35, 0x27, 0x91, 0xc7, 0xb3, 0xdd, 0xb0, 0x92, 0x38, 0xf4, - 0xf1, 0xa1, 0x9d, 0x53, 0xf2, 0x78, 0xc6, 0x81, 0x54, 0xb8, 0xc4, 0xfa, 0x8a, 0x9f, 0x07, 0xb6, - 0xf8, 0x11, 0xd6, 0xd2, 0x0d, 0x2a, 0x6a, 0xe8, 0xb6, 0x52, 0xe4, 0x31, 0xe5, 0xda, 0x84, 0x34, - 0xa7, 0x0c, 0xde, 0x25, 0x13, 0x9e, 0x2f, 0x3d, 0x6c, 0x33, 0xaa, 0x74, 0x73, 0xe8, 0x84, 0x8f, - 0xd8, 0xc3, 0x23, 0x13, 0x5e, 0x09, 0x90, 0xd9, 0x02, 0xa4, 0xc9, 0xb6, 0xcd, 0x5e, 0xb8, 0xdf, - 0x1a, 0xb1, 0x00, 0xf5, 0xed, 0x02, 0xb0, 0x05, 0x48, 0x6b, 0x30, 0x4e, 0x22, 0x48, 0x6d, 0x63, - 0xc5, 0xe2, 0x66, 0xf6, 0xf6, 0x50, 0x41, 0x03, 0xf9, 0xf3, 0x88, 0x20, 0xd5, 0x23, 0x12, 0x87, - 0xc7, 0x72, 0x93, 0xbc, 0x70, 0x87, 0xf1, 0xce, 0x50, 0x87, 0x27, 0x32, 0x0b, 0x0d, 0x71, 0x78, - 0xac, 0xd0, 0x0d, 0xf4, 0x73, 0x30, 0xcd, 0xe1, 0x5b, 0xe9, 0xee, 0x08, 0x37, 0x36, 0x88, 0xb8, - 0xc9, 0xbc, 0xe6, 0x3c, 0xcc, 0xca, 0x32, 0xd8, 0xc8, 0x9a, 0x77, 0x6f, 0x84, 0x95, 0x1d, 0x40, - 0xae, 0xcc, 0xca, 0xfa, 0x64, 0x62, 0x65, 0xd9, 0x38, 0xe5, 0x6b, 0xdd, 0xfd, 0xa1, 0x56, 0x76, - 0xf0, 0xe8, 0x0b, 0xb1, 0xb2, 0xaf, 0x7d, 0x2a, 0x69, 0x99, 0xcd, 0xe0, 0x53, 0xe9, 0x1b, 0x43, - 0x5b, 0x16, 0xc6, 0x91, 0xa4, 0x65, 0x9c, 0x87, 0x74, 0x1b, 0x73, 0x89, 0x99, 0xa6, 0xdf, 0x1d, - 0x7e, 0xe8, 0xbe, 0x1f, 0x74, 0xd4, 0xdc, 0x8d, 0x4a, 0xa6, 0x61, 0xcf, 0x50, 0x59, 0xfc, 0xd0, - 0x31, 0xd7, 0xd4, 0x7b, 0xa3, 0x0d, 0x55, 0xd4, 0x49, 0x6a, 0xcf, 0x50, 0x85, 0x6e, 0xd2, 0xaa, - 0xb2, 0x73, 0x66, 0x74, 0x7e, 0x2f, 0x8f, 0xc8, 0x0f, 0xd0, 0x77, 0xda, 0x8f, 0x56, 0xd5, 0x23, - 0xfa, 0x53, 0xa8, 0xc7, 0x92, 0x57, 0x94, 0x56, 0x46, 0x4f, 0xa1, 0x70, 0xfa, 0x0c, 0x6f, 0x0a, - 0x71, 0xb2, 0xb7, 0x66, 0xba, 0x1e, 0xc6, 0xfb, 0xa3, 0xd7, 0xcc, 0x7e, 0xd7, 0x82, 0xad, 0x99, - 0xdc, 0xa7, 0xf8, 0xeb, 0x02, 0x2c, 0xb1, 0xba, 0xd1, 0x1d, 0xbd, 0x13, 0xd9, 0xdb, 0x17, 0x0d, - 0x9c, 0x79, 0x78, 0x40, 0x1f, 0xf0, 0xe1, 0xb0, 0xea, 0x9e, 0xb3, 0xcf, 0x5b, 0x9b, 0x90, 0xde, - 0x51, 0x46, 0x95, 0x2b, 0x4f, 0xf3, 0xd7, 0xa2, 0xde, 0x21, 0xcf, 0x42, 0xb1, 0xb8, 0x9e, 0x4a, - 0x5f, 0x29, 0x96, 0xd6, 0x53, 0xe9, 0xab, 0xc5, 0x85, 0xf5, 0x54, 0xfa, 0xad, 0xe2, 0xdb, 0xe2, - 0x5f, 0x5e, 0x85, 0x9c, 0x8b, 0xfc, 0x18, 0x22, 0x7a, 0x18, 0x44, 0x44, 0xd7, 0x86, 0x21, 0x22, - 0x8e, 0x15, 0x39, 0x24, 0x7a, 0x18, 0x84, 0x44, 0xd7, 0x86, 0x41, 0x22, 0x9f, 0x87, 0x60, 0xa2, - 0xe6, 0x30, 0x4c, 0x74, 0x6f, 0x0c, 0x4c, 0xe4, 0x89, 0xea, 0x07, 0x45, 0x6b, 0x83, 0xa0, 0xe8, - 0xe6, 0x68, 0x50, 0xe4, 0x89, 0x0a, 0xa0, 0xa2, 0xc7, 0x7d, 0xa8, 0xe8, 0xfa, 0x08, 0x54, 0xe4, - 0xf1, 0xbb, 0xb0, 0x68, 0x23, 0x12, 0x16, 0xdd, 0x3e, 0x0f, 0x16, 0x79, 0x72, 0x42, 0xb8, 0xe8, - 0x83, 0x10, 0x2e, 0x5a, 0x1c, 0x8a, 0x8b, 0x3c, 0x6e, 0x06, 0x8c, 0x3e, 0xe9, 0x07, 0x46, 0xd7, - 0x47, 0x00, 0x23, 0xbf, 0x05, 0x1c, 0x19, 0xd5, 0xa2, 0x90, 0xd1, 0xad, 0x73, 0x90, 0x91, 0x27, - 0x25, 0x08, 0x8d, 0x6a, 0x51, 0xd0, 0xe8, 0xd6, 0x39, 0xd0, 0xa8, 0x4f, 0x12, 0xc3, 0x46, 0x5b, - 0xd1, 0xd8, 0xe8, 0xce, 0xb9, 0xd8, 0xc8, 0x93, 0x16, 0x06, 0x47, 0x2b, 0x01, 0x70, 0xf4, 0xce, - 0x10, 0x70, 0xe4, 0xb1, 0x12, 0x74, 0xf4, 0xad, 0x01, 0x74, 0x24, 0x8e, 0x42, 0x47, 0x1e, 0xaf, - 0x07, 0x8f, 0x5e, 0x0c, 0x81, 0x47, 0x77, 0xcf, 0x87, 0x47, 0x9e, 0xb0, 0x3e, 0x7c, 0xa4, 0x8c, - 0xc4, 0x47, 0xef, 0x8d, 0x89, 0x8f, 0x3c, 0xe9, 0x51, 0x00, 0xe9, 0xa3, 0x30, 0x40, 0x5a, 0x1a, - 0x0e, 0x90, 0x3c, 0x31, 0x1c, 0x21, 0x6d, 0x44, 0x22, 0xa4, 0xdb, 0xe7, 0x21, 0x24, 0x7f, 0x1e, - 0x04, 0x21, 0xd2, 0x56, 0x34, 0x44, 0xba, 0x73, 0x2e, 0x44, 0xf2, 0xbb, 0x3f, 0x84, 0x91, 0x36, - 0x22, 0x31, 0xd2, 0xed, 0xf3, 0x30, 0x92, 0x5f, 0xb9, 0x20, 0x48, 0x7a, 0x35, 0x14, 0x24, 0xdd, - 0x1f, 0x07, 0x24, 0x79, 0x42, 0x07, 0x50, 0xd2, 0xe7, 0xc3, 0x51, 0xd2, 0x37, 0x2e, 0x90, 0xaf, - 0x30, 0x12, 0x26, 0x7d, 0x6b, 0x00, 0x26, 0x89, 0xa3, 0x60, 0x92, 0x3f, 0x9e, 0x5d, 0x9c, 0xa4, - 0x8c, 0x44, 0x35, 0xef, 0x8d, 0x89, 0x6a, 0xfc, 0xc1, 0x17, 0x01, 0x6b, 0xaa, 0x11, 0xb0, 0xe6, - 0xe6, 0x68, 0x58, 0xe3, 0x9b, 0x73, 0x1f, 0xd7, 0xd4, 0xa2, 0x70, 0xcd, 0xad, 0x73, 0x70, 0x8d, - 0x6f, 0x85, 0x02, 0xc0, 0xe6, 0x71, 0x1f, 0xb0, 0xb9, 0x7e, 0x6e, 0xd4, 0x4f, 0x00, 0xd9, 0x94, - 0x07, 0x91, 0xcd, 0x8d, 0x91, 0xc8, 0xc6, 0x93, 0xe0, 0x43, 0x9b, 0xc7, 0x7d, 0xd0, 0xe6, 0xfa, - 0x08, 0x68, 0xe3, 0x57, 0x80, 0x63, 0x1b, 0x6d, 0x34, 0xb6, 0x59, 0x1e, 0x17, 0xdb, 0x78, 0x82, - 0x23, 0xc1, 0xcd, 0x56, 0x34, 0xb8, 0xb9, 0x33, 0xe6, 0x0b, 0xf9, 0x01, 0x74, 0x53, 0x8b, 0x42, - 0x37, 0xb7, 0xce, 0x41, 0x37, 0xc1, 0x35, 0xc4, 0x83, 0x37, 0xb5, 0x28, 0x78, 0x73, 0xeb, 0x1c, - 0x78, 0xe3, 0x4b, 0x0a, 0xe0, 0x9b, 0xe6, 0x30, 0x7c, 0x73, 0x6f, 0x0c, 0x7c, 0xe3, 0x3b, 0x2f, - 0x7d, 0x00, 0xe7, 0xd3, 0x7e, 0x80, 0x23, 0x8e, 0x02, 0x38, 0xfe, 0x8c, 0x74, 0x11, 0xce, 0x56, - 0x34, 0xc2, 0xb9, 0x73, 0x2e, 0xc2, 0x09, 0x1a, 0xc9, 0x00, 0xc4, 0xd9, 0x88, 0x84, 0x38, 0xb7, - 0xcf, 0x83, 0x38, 0xbe, 0x91, 0x0c, 0x62, 0x9c, 0x4f, 0xfb, 0x31, 0x8e, 0x38, 0x0a, 0xe3, 0xf8, - 0x8d, 0x73, 0x41, 0x4e, 0x2d, 0x0a, 0xe4, 0xdc, 0x3a, 0x07, 0xe4, 0xf8, 0x9d, 0x17, 0x40, 0x39, - 0xca, 0x48, 0x94, 0xf3, 0xde, 0x98, 0x28, 0xa7, 0xcf, 0x70, 0x85, 0x61, 0x4e, 0x2d, 0x0a, 0xe6, - 0xdc, 0x3a, 0x07, 0xe6, 0x04, 0x2a, 0xeb, 0xe3, 0x9c, 0xad, 0x68, 0x9c, 0x73, 0xe7, 0x5c, 0x9c, - 0xd3, 0x37, 0x9b, 0x5c, 0xa0, 0xb3, 0x11, 0x09, 0x74, 0x6e, 0x9f, 0x07, 0x74, 0xfa, 0x16, 0x3e, - 0xee, 0x1c, 0xfc, 0xca, 0xf8, 0x48, 0xe7, 0xa3, 0x8b, 0x23, 0x1d, 0xef, 0x99, 0xb1, 0x40, 0x9d, - 0xf5, 0x54, 0xfa, 0xed, 0xe2, 0x3b, 0xe2, 0xdf, 0x9a, 0x86, 0xa9, 0x9a, 0x17, 0xe7, 0xe2, 0xd7, - 0x52, 0x78, 0x93, 0xbc, 0x48, 0x68, 0x8d, 0xcc, 0x58, 0x6a, 0xf7, 0xce, 0x4f, 0x75, 0x37, 0x98, - 0x8c, 0x8d, 0xb3, 0xbe, 0xc1, 0x01, 0x65, 0xf4, 0x01, 0xe4, 0x7a, 0x36, 0xb6, 0xe4, 0xae, 0xa5, - 0x9b, 0x96, 0xee, 0xb0, 0x53, 0x1c, 0x42, 0xb9, 0xf8, 0xe5, 0xe9, 0xe2, 0xcc, 0x8e, 0x8d, 0xad, - 0x6d, 0x4e, 0x97, 0x66, 0x7a, 0x81, 0x2b, 0xf7, 0x43, 0x51, 0x93, 0xe3, 0x7f, 0x28, 0xea, 0x05, - 0x14, 0x2d, 0xac, 0x68, 0x21, 0x0f, 0x84, 0xe5, 0x20, 0x8a, 0x1e, 0x33, 0xf4, 0x94, 0x95, 0x5b, - 0x92, 0xe6, 0x22, 0x2a, 0x58, 0x61, 0x22, 0x7a, 0x00, 0x97, 0x3a, 0xca, 0x31, 0x8d, 0x89, 0x94, - 0x5d, 0xa7, 0x8e, 0xc6, 0x39, 0xb2, 0x6f, 0x30, 0xa1, 0x8e, 0x72, 0x4c, 0xbf, 0x3a, 0xc5, 0x6e, - 0xd1, 0x4f, 0x49, 0xdc, 0x82, 0xbc, 0xa6, 0xdb, 0x8e, 0x6e, 0xa8, 0x0e, 0xcf, 0x3d, 0xcb, 0xf2, - 0xb6, 0xe6, 0x5c, 0x2a, 0x4b, 0x30, 0x7b, 0x1f, 0x66, 0x79, 0xb0, 0x7c, 0xe0, 0x15, 0x21, 0xcf, - 0xdf, 0xca, 0x6e, 0x78, 0x6f, 0x05, 0x51, 0x05, 0x0a, 0x2d, 0xc5, 0xc1, 0x47, 0xca, 0x89, 0xec, - 0x1e, 0xc5, 0xca, 0xd2, 0x94, 0x8c, 0x6f, 0x9d, 0x9d, 0x2e, 0xe6, 0x9e, 0xb2, 0x5b, 0x03, 0x27, - 0xb2, 0x72, 0xad, 0xc0, 0x0d, 0x0d, 0xdd, 0x81, 0x82, 0x62, 0x9f, 0x18, 0x2a, 0x55, 0x0f, 0x36, - 0xec, 0x9e, 0x4d, 0x21, 0x45, 0x5a, 0xca, 0x53, 0x72, 0xc5, 0xa5, 0xa2, 0xeb, 0x30, 0xc3, 0x23, - 0xc9, 0xd9, 0x67, 0x6c, 0x0a, 0xb4, 0xa9, 0xfc, 0xbb, 0x09, 0xf4, 0x4b, 0x36, 0xe8, 0x31, 0x2c, - 0xf0, 0xdc, 0xf2, 0x47, 0x8a, 0xa5, 0xc9, 0x54, 0xeb, 0xfe, 0xf8, 0x2c, 0x52, 0xb1, 0x57, 0x58, - 0x2e, 0x79, 0x52, 0x80, 0xa8, 0xda, 0xcf, 0x84, 0xb0, 0x05, 0xb3, 0x6a, 0x5b, 0xf7, 0x10, 0x00, - 0x6b, 0xf9, 0xec, 0x50, 0x3b, 0x5b, 0xa1, 0x65, 0xfd, 0x57, 0xa4, 0x05, 0x35, 0x4c, 0x40, 0x0d, - 0xa0, 0xd9, 0x5e, 0xe4, 0xae, 0xd9, 0xd6, 0xd5, 0x13, 0xea, 0xfc, 0x87, 0x73, 0x64, 0x8f, 0xcc, - 0x54, 0xff, 0x4a, 0xd1, 0x9d, 0x6d, 0xca, 0x29, 0xc1, 0x91, 0xf7, 0x9b, 0xe5, 0xb7, 0x5d, 0x4f, - 0xa5, 0x67, 0x8a, 0xb9, 0xf5, 0x54, 0x3a, 0x5f, 0x2c, 0x88, 0x7f, 0x5b, 0x80, 0x42, 0x5f, 0x5d, - 0x50, 0x0d, 0x2e, 0x69, 0xde, 0x54, 0x91, 0xf9, 0x51, 0x23, 0xdd, 0x34, 0x78, 0xda, 0xef, 0xb9, - 0x2f, 0x4f, 0x17, 0x0b, 0xb4, 0xf4, 0x53, 0xef, 0x96, 0x34, 0xef, 0x73, 0xf8, 0x54, 0xf4, 0x11, - 0xe4, 0x99, 0xfb, 0xe8, 0x7d, 0x99, 0x8d, 0xc6, 0x88, 0x97, 0x67, 0xbf, 0x3c, 0x5d, 0xcc, 0x51, - 0x9f, 0xd1, 0x4d, 0xcb, 0x2b, 0xe5, 0xda, 0xc1, 0x4b, 0xf1, 0x37, 0x04, 0x98, 0x09, 0x1d, 0xe7, - 0x79, 0xdc, 0xf7, 0x06, 0xfd, 0x6a, 0x34, 0xee, 0x1c, 0x16, 0x50, 0x97, 0xe6, 0xe3, 0xdc, 0x8d, - 0x4e, 0x5c, 0x1c, 0x8e, 0x5b, 0xe8, 0x2e, 0x8c, 0x1b, 0xaa, 0xe1, 0xb2, 0x7d, 0x9c, 0xfa, 0xfe, - 0x0f, 0x16, 0x27, 0xc4, 0x3f, 0x48, 0x41, 0x2e, 0x7c, 0x78, 0xa7, 0xde, 0x57, 0xaf, 0xa8, 0x75, - 0x21, 0xc4, 0xb1, 0x3c, 0x22, 0x6d, 0x61, 0xc6, 0xcf, 0xaf, 0xcf, 0xaa, 0xb9, 0x34, 0x22, 0x4e, - 0x20, 0x58, 0x4f, 0x9f, 0x71, 0xe1, 0x7b, 0x49, 0xcf, 0xbe, 0x2e, 0xc3, 0x24, 0x4d, 0xa3, 0xc3, - 0xab, 0x56, 0xea, 0x1f, 0x3d, 0xc4, 0x57, 0x26, 0xf7, 0x25, 0x56, 0x8c, 0xd8, 0xe3, 0xe6, 0x1b, - 0xe5, 0xa9, 0xf3, 0xa7, 0xc1, 0xc5, 0x3f, 0x84, 0xc7, 0xf3, 0x14, 0x4e, 0x5e, 0x2c, 0x4f, 0x21, - 0xfa, 0x25, 0x28, 0xa8, 0x66, 0xbb, 0xcd, 0xd6, 0x3a, 0x66, 0x91, 0x06, 0x33, 0x8f, 0x50, 0x11, - 0xfc, 0xdb, 0x85, 0xcb, 0xde, 0x37, 0x0c, 0x97, 0x25, 0xfe, 0x0d, 0xc3, 0x40, 0xac, 0x67, 0xde, - 0x13, 0xc6, 0x0c, 0x59, 0x5f, 0xd8, 0xe9, 0xf4, 0x9b, 0x84, 0x9d, 0xb2, 0x50, 0x65, 0x3e, 0x72, - 0xfe, 0x54, 0xe0, 0x81, 0x21, 0xcf, 0x4c, 0xf3, 0xa0, 0xe7, 0x85, 0x8b, 0x2e, 0x04, 0xb3, 0x06, - 0xa6, 0xbf, 0x3c, 0x5d, 0x4c, 0x49, 0x5e, 0xda, 0xc0, 0x28, 0xcb, 0x9f, 0xf8, 0xc9, 0x2c, 0xff, - 0x75, 0x98, 0xe9, 0x5a, 0x78, 0x0f, 0x3b, 0xea, 0xbe, 0x6c, 0xf4, 0x3a, 0xfc, 0x54, 0x49, 0xd6, - 0xa5, 0x6d, 0xf5, 0x3a, 0xe8, 0x1e, 0x14, 0xbd, 0x22, 0x1c, 0x63, 0xbb, 0x69, 0xa5, 0x5c, 0x3a, - 0x47, 0xe4, 0xe2, 0xff, 0x16, 0x60, 0x2e, 0xd4, 0x26, 0x3e, 0x27, 0xd6, 0x21, 0xeb, 0x9b, 0x03, - 0xbb, 0x24, 0x5c, 0x30, 0x78, 0x32, 0xc8, 0x8c, 0x64, 0xb8, 0xec, 0x3e, 0x96, 0xe6, 0x94, 0xf7, - 0xc5, 0x26, 0x2e, 0x28, 0xf6, 0x92, 0x2f, 0x67, 0x2d, 0xf0, 0x00, 0x6f, 0x92, 0x24, 0xc7, 0x9a, - 0x24, 0xe2, 0x6f, 0x0b, 0x50, 0xa4, 0x0f, 0x78, 0x82, 0xb1, 0x16, 0x8b, 0x75, 0x72, 0x83, 0x92, - 0x13, 0xe3, 0x9f, 0x1b, 0x09, 0x7d, 0x07, 0x23, 0x19, 0xfe, 0x0e, 0x86, 0xf8, 0x03, 0x01, 0xf2, - 0x5e, 0x0d, 0xd9, 0xd7, 0xe0, 0x46, 0x24, 0xa7, 0x7c, 0xb3, 0x2f, 0xa1, 0xb9, 0x59, 0x38, 0xc6, - 0xfa, 0x40, 0x5d, 0x30, 0x0b, 0x07, 0xfb, 0x82, 0xd7, 0xdf, 0x75, 0x47, 0x0e, 0xa9, 0x62, 0xc5, - 0xcf, 0xb6, 0xf0, 0x06, 0x47, 0x68, 0x24, 0xfa, 0x21, 0x4d, 0xb3, 0x7d, 0xc8, 0x12, 0x9f, 0x8c, - 0x65, 0xb6, 0x10, 0x0f, 0x87, 0x02, 0xbe, 0x1b, 0xa7, 0x35, 0x1b, 0xf4, 0x13, 0x9b, 0xec, 0xb7, - 0x2d, 0x3e, 0x09, 0x28, 0x90, 0x76, 0x3e, 0xd1, 0xd2, 0x58, 0xa6, 0xd4, 0xd5, 0x12, 0x1b, 0x2b, - 0x7f, 0x12, 0xec, 0x89, 0xea, 0x21, 0x41, 0x61, 0x8f, 0x20, 0x79, 0xa8, 0xb4, 0x47, 0x85, 0x81, - 0x85, 0x7a, 0x4e, 0x22, 0xa5, 0xd1, 0x93, 0x50, 0x92, 0x8a, 0xc4, 0x70, 0xc4, 0x30, 0xa8, 0xd2, - 0x50, 0x32, 0x8b, 0x0f, 0xc3, 0x63, 0x7d, 0xe4, 0xe3, 0x83, 0x83, 0xfe, 0xe3, 0xd4, 0x0f, 0x7f, - 0xb0, 0x28, 0x88, 0x9f, 0x00, 0x92, 0xb0, 0x8d, 0x9d, 0x17, 0x3d, 0xd3, 0xf2, 0x13, 0x7e, 0xdc, - 0xee, 0xfb, 0x30, 0xc8, 0x64, 0x39, 0x7b, 0x16, 0x95, 0xd2, 0xe7, 0x12, 0xcc, 0x85, 0xb8, 0x99, - 0xb1, 0x10, 0x3f, 0x84, 0xab, 0x4f, 0x4d, 0xdb, 0xd6, 0xbb, 0x04, 0x7a, 0xd2, 0x59, 0x49, 0x96, - 0x06, 0xcf, 0x3c, 0xa6, 0xbb, 0x74, 0x13, 0xc2, 0x60, 0x66, 0x24, 0x23, 0x79, 0xd7, 0xe2, 0x1f, - 0x0a, 0x70, 0x65, 0x90, 0x93, 0x69, 0x39, 0xea, 0xc4, 0xdf, 0xb4, 0x6a, 0xfa, 0xf9, 0xdd, 0xce, - 0x1f, 0xad, 0x6e, 0x71, 0xe2, 0x62, 0xf2, 0x67, 0xca, 0x1d, 0x85, 0x9a, 0x0f, 0x7e, 0xfa, 0x38, - 0xcf, 0xc9, 0x9b, 0x8c, 0xea, 0x5b, 0x92, 0xd4, 0x78, 0x96, 0xa4, 0x09, 0x85, 0x75, 0x53, 0x37, - 0x88, 0x27, 0xeb, 0xb6, 0x77, 0x15, 0xf2, 0xbb, 0xba, 0xa1, 0x58, 0x27, 0xb2, 0x7b, 0xa8, 0x9b, - 0x0d, 0x94, 0x85, 0xa8, 0xca, 0xb2, 0x12, 0x52, 0x8e, 0x71, 0xf0, 0x4b, 0xf1, 0x47, 0x02, 0x14, - 0x7d, 0xb1, 0xdc, 0x22, 0xbf, 0x0b, 0xa0, 0xb6, 0x7b, 0xb6, 0x83, 0x2d, 0xb7, 0x97, 0x66, 0x58, - 0xf4, 0x76, 0x85, 0x51, 0xeb, 0x6b, 0x52, 0x86, 0x17, 0xa8, 0x6b, 0xe8, 0x46, 0x38, 0x39, 0xc2, - 0x64, 0x19, 0xce, 0x06, 0x52, 0x22, 0x90, 0x6e, 0xb7, 0x1d, 0xd3, 0xf2, 0x30, 0x13, 0xef, 0x76, - 0x37, 0x95, 0x3a, 0x3d, 0xd3, 0x4b, 0xca, 0xad, 0x42, 0x9e, 0x2c, 0xf7, 0x87, 0xd8, 0x6b, 0x52, - 0xea, 0xfc, 0x26, 0x31, 0x0e, 0xb7, 0x49, 0xbf, 0x47, 0x9c, 0x54, 0xd6, 0x1b, 0x5e, 0x0f, 0x8f, - 0xb0, 0x68, 0x3f, 0x13, 0x4c, 0x49, 0x38, 0x5e, 0xd2, 0x46, 0xea, 0x89, 0x7c, 0x0b, 0xd2, 0xee, - 0x27, 0x8a, 0xf9, 0x04, 0xb9, 0xba, 0xcc, 0xbe, 0x61, 0xbc, 0xec, 0x7e, 0xc3, 0x78, 0x79, 0x8d, - 0x17, 0x60, 0x66, 0xfc, 0xfb, 0xff, 0x6d, 0x51, 0x90, 0x3c, 0xa6, 0xfb, 0x0d, 0x32, 0xc2, 0x07, - 0x56, 0x61, 0x94, 0x07, 0x08, 0x7c, 0x43, 0x86, 0x7f, 0xb5, 0x76, 0x75, 0x4d, 0xde, 0xd9, 0xaa, - 0x3c, 0xdf, 0xdc, 0xac, 0x37, 0x9b, 0xd5, 0xb5, 0xa2, 0x80, 0x8a, 0x30, 0x13, 0xfa, 0x02, 0x4d, - 0x82, 0x7d, 0xc7, 0xf6, 0xfe, 0xcf, 0x00, 0xf8, 0x1f, 0xb3, 0x22, 0xb2, 0x36, 0xaa, 0x9f, 0xc9, - 0x2f, 0x57, 0x9f, 0xed, 0x54, 0x1b, 0xc5, 0x09, 0x84, 0x20, 0x5f, 0x5e, 0x6d, 0x56, 0x6a, 0xb2, - 0x54, 0x6d, 0x6c, 0x3f, 0xdf, 0x6a, 0x54, 0xdd, 0xef, 0xdf, 0xde, 0x5f, 0x83, 0x99, 0x60, 0xba, - 0x1b, 0x34, 0x07, 0x85, 0x4a, 0xad, 0x5a, 0xd9, 0x90, 0x5f, 0xd6, 0x57, 0xe5, 0x17, 0x3b, 0xd5, - 0x9d, 0x6a, 0x71, 0x82, 0x56, 0x8d, 0x12, 0x9f, 0xec, 0x3c, 0x7b, 0x56, 0x14, 0x50, 0x01, 0xb2, - 0xec, 0x9a, 0x7e, 0xad, 0xa6, 0x98, 0xb8, 0xbf, 0x09, 0xd9, 0x40, 0x5a, 0x5b, 0xf2, 0xb8, 0xed, - 0x9d, 0x46, 0x4d, 0x6e, 0xd6, 0x37, 0xab, 0x8d, 0xe6, 0xea, 0xe6, 0x36, 0x93, 0x41, 0x69, 0xab, - 0xe5, 0xe7, 0x52, 0xb3, 0x28, 0x78, 0xd7, 0xcd, 0xe7, 0x3b, 0x95, 0x9a, 0xdb, 0x0c, 0x31, 0x95, - 0x4e, 0x16, 0x93, 0xf7, 0xff, 0x9a, 0x00, 0x57, 0x86, 0xa4, 0x7e, 0x41, 0x59, 0x98, 0xde, 0x31, - 0x68, 0x8e, 0xcf, 0xe2, 0x04, 0xca, 0x05, 0xb2, 0xbf, 0x14, 0x05, 0x94, 0x66, 0xf9, 0x37, 0x8a, - 0x09, 0x34, 0x05, 0x89, 0xc6, 0xa3, 0x62, 0x92, 0xd4, 0x34, 0x90, 0x3c, 0xa5, 0x98, 0x42, 0x19, - 0x9e, 0xb6, 0xa1, 0x38, 0x89, 0x66, 0xfc, 0xec, 0x09, 0xc5, 0x29, 0x22, 0xca, 0xcb, 0x42, 0x50, - 0x9c, 0xbe, 0x7f, 0x1d, 0x02, 0x27, 0xbd, 0x11, 0xc0, 0xd4, 0x33, 0xc5, 0xc1, 0xb6, 0x53, 0x9c, - 0x40, 0xd3, 0x90, 0x5c, 0x6d, 0xb7, 0x8b, 0xc2, 0xc3, 0x7f, 0x9d, 0x82, 0xb4, 0xfb, 0x55, 0x16, - 0xf4, 0x0c, 0x26, 0xd9, 0xd6, 0xf3, 0xe2, 0x70, 0xcf, 0x9e, 0x4e, 0xde, 0x85, 0xa5, 0xf3, 0x5c, - 0x7f, 0x71, 0x02, 0xfd, 0x15, 0xc8, 0x06, 0x3c, 0x26, 0x34, 0x74, 0xfb, 0x2c, 0xe4, 0x25, 0x2e, - 0xdc, 0x3e, 0xaf, 0x98, 0x27, 0xff, 0x15, 0x64, 0x3c, 0x0b, 0x8e, 0x6e, 0x8c, 0xb2, 0xef, 0xae, - 0xec, 0xd1, 0x8b, 0x00, 0x99, 0x6b, 0xe2, 0xc4, 0xfb, 0x02, 0xb2, 0x00, 0x0d, 0x1a, 0x5b, 0x14, - 0x15, 0x91, 0x30, 0xd4, 0x9a, 0x2f, 0xdc, 0x1f, 0xab, 0xb4, 0xff, 0x4c, 0xa2, 0x2c, 0x7f, 0xc5, - 0x88, 0x56, 0xd6, 0xc0, 0x7a, 0x14, 0xad, 0xac, 0x88, 0x85, 0x67, 0x02, 0xbd, 0x80, 0x14, 0xb1, - 0x94, 0x28, 0xca, 0x87, 0xec, 0xb3, 0xcc, 0x0b, 0x37, 0x46, 0x96, 0x71, 0x45, 0x96, 0xef, 0xfd, - 0xf0, 0x2f, 0xae, 0x4d, 0xfc, 0xf0, 0xec, 0x9a, 0xf0, 0xa3, 0xb3, 0x6b, 0xc2, 0x9f, 0x9d, 0x5d, - 0x13, 0xfe, 0xfc, 0xec, 0x9a, 0xf0, 0xeb, 0x3f, 0xbe, 0x36, 0xf1, 0xa3, 0x1f, 0x5f, 0x9b, 0xf8, - 0xb3, 0x1f, 0x5f, 0x9b, 0xf8, 0x7c, 0x9a, 0x73, 0xef, 0x4e, 0x51, 0xa3, 0xf2, 0xe8, 0xff, 0x05, - 0x00, 0x00, 0xff, 0xff, 0x2a, 0xe5, 0xf1, 0xd1, 0x53, 0x7e, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_0db44e1715aa2d9a) } + +var fileDescriptor_api_0db44e1715aa2d9a = []byte{ + // 8186 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0xbd, 0x6b, 0x6c, 0x23, 0x59, + 0x76, 0x1f, 0xae, 0x22, 0x29, 0x89, 0x3c, 0x14, 0x1f, 0xba, 0xea, 0x07, 0x9b, 0x33, 0xd3, 0xea, + 0xae, 0x7e, 0xf7, 0xce, 0x48, 0xd3, 0xdd, 0x3b, 0x9e, 0xf1, 0xf4, 0x78, 0xd6, 0x12, 0xc5, 0x6e, + 0x52, 0x6a, 0xa9, 0xd5, 0x45, 0xaa, 0xdb, 0x33, 0x5e, 0xff, 0x6b, 0x4b, 0x55, 0x57, 0x54, 0xad, + 0xc8, 0x2a, 0x76, 0x55, 0x51, 0x8f, 0x01, 0xfe, 0x40, 0x1c, 0x1b, 0x89, 0x81, 0x00, 0x8b, 0xfd, + 0xe0, 0x20, 0x6b, 0x38, 0x89, 0xd7, 0x71, 0x1c, 0x07, 0x08, 0xf2, 0x00, 0x12, 0xc4, 0x41, 0x90, + 0xd8, 0x5f, 0x8c, 0x64, 0x11, 0x38, 0xc8, 0xfa, 0x53, 0x8c, 0x00, 0x51, 0x6c, 0x6d, 0xbe, 0x25, + 0x08, 0x82, 0x7c, 0x09, 0x30, 0x1f, 0x82, 0xe0, 0x3e, 0xea, 0x45, 0x16, 0x1f, 0xea, 0xad, 0x41, + 0x06, 0xf0, 0x17, 0x89, 0x75, 0xea, 0x9e, 0x53, 0xf7, 0x9e, 0x7b, 0xef, 0xb9, 0xe7, 0x77, 0xeb, + 0xdc, 0x53, 0x30, 0x6f, 0x99, 0x8a, 0xba, 0xdf, 0xdd, 0x5d, 0x56, 0xba, 0xfa, 0x52, 0xd7, 0x32, + 0x1d, 0x13, 0xcd, 0xab, 0xa6, 0x7a, 0x40, 0xc9, 0x4b, 0xfc, 0x66, 0xf9, 0xfe, 0xc1, 0xe1, 0xf2, + 0xc1, 0xa1, 0x8d, 0xad, 0x43, 0x6c, 0x2d, 0xab, 0xa6, 0xa1, 0xf6, 0x2c, 0x0b, 0x1b, 0xea, 0xc9, + 0x72, 0xdb, 0x54, 0x0f, 0xe8, 0x1f, 0xdd, 0x68, 0x31, 0xf6, 0x32, 0x72, 0x25, 0x6a, 0x8a, 0xa3, + 0x70, 0xda, 0x05, 0x97, 0x86, 0x2d, 0xcb, 0xb4, 0x6c, 0x4e, 0xbd, 0xe4, 0x52, 0x3b, 0xd8, 0x51, + 0x02, 0xa5, 0xdf, 0xb2, 0x1d, 0xd3, 0x52, 0x5a, 0x78, 0x19, 0x1b, 0x2d, 0xdd, 0xc0, 0xa4, 0xc0, + 0xa1, 0xaa, 0xf2, 0x9b, 0x6f, 0x47, 0xde, 0x7c, 0xc4, 0xef, 0x96, 0x7a, 0x8e, 0xde, 0x5e, 0xde, + 0x6f, 0xab, 0xcb, 0x8e, 0xde, 0xc1, 0xb6, 0xa3, 0x74, 0xba, 0xfc, 0xce, 0x7d, 0x7a, 0xc7, 0xb1, + 0x14, 0x55, 0x37, 0x5a, 0xee, 0xff, 0xee, 0xee, 0xb2, 0x85, 0x55, 0xd3, 0xd2, 0xb0, 0x26, 0xdb, + 0x5d, 0xc5, 0x70, 0xab, 0xdb, 0x32, 0x5b, 0x26, 0xfd, 0xb9, 0x4c, 0x7e, 0x71, 0xea, 0xd5, 0x96, + 0x69, 0xb6, 0xda, 0x78, 0x99, 0x5e, 0xed, 0xf6, 0xf6, 0x96, 0xb5, 0x9e, 0xa5, 0x38, 0xba, 0xc9, + 0xb9, 0xc4, 0x7f, 0x2e, 0x40, 0x4e, 0xc2, 0xaf, 0x7b, 0xd8, 0x76, 0x6a, 0x58, 0xd1, 0xb0, 0x85, + 0xae, 0x40, 0xf2, 0x00, 0x9f, 0x94, 0x92, 0xd7, 0x84, 0xbb, 0x73, 0xab, 0xb3, 0x5f, 0x9e, 0x2e, + 0x26, 0x37, 0xf0, 0x89, 0x44, 0x68, 0xe8, 0x1a, 0xcc, 0x62, 0x43, 0x93, 0xc9, 0xed, 0x54, 0xf8, + 0xf6, 0x0c, 0x36, 0xb4, 0x0d, 0x7c, 0x82, 0xbe, 0x0d, 0x69, 0x9b, 0x48, 0x33, 0x54, 0x5c, 0x9a, + 0xbe, 0x26, 0xdc, 0x9d, 0x5e, 0xfd, 0xf9, 0x2f, 0x4f, 0x17, 0x3f, 0x69, 0xe9, 0xce, 0x7e, 0x6f, + 0x77, 0x49, 0x35, 0x3b, 0xcb, 0x5e, 0x3f, 0x69, 0xbb, 0xfe, 0xef, 0xe5, 0xee, 0x41, 0x6b, 0xb9, + 0x5f, 0x47, 0x4b, 0xcd, 0x63, 0xa3, 0x81, 0x5f, 0x4b, 0x9e, 0xc4, 0xf5, 0x54, 0x5a, 0x28, 0x26, + 0xd6, 0x53, 0xe9, 0x44, 0x31, 0x29, 0xfe, 0x4e, 0x12, 0xf2, 0x12, 0xb6, 0xbb, 0xa6, 0x61, 0x63, + 0x5e, 0xf3, 0xf7, 0x21, 0xe9, 0x1c, 0x1b, 0xb4, 0xe6, 0xd9, 0x87, 0x57, 0x97, 0x06, 0x46, 0xc4, + 0x52, 0xd3, 0x52, 0x0c, 0x5b, 0x51, 0x49, 0xf3, 0x25, 0x52, 0x14, 0x7d, 0x04, 0x59, 0x0b, 0xdb, + 0xbd, 0x0e, 0xa6, 0x8a, 0xa4, 0x8d, 0xca, 0x3e, 0xbc, 0x1c, 0xc1, 0xd9, 0xe8, 0x2a, 0x86, 0x04, + 0xac, 0x2c, 0xf9, 0x8d, 0xae, 0x40, 0xda, 0xe8, 0x75, 0x88, 0x2a, 0x6c, 0xda, 0xd0, 0xa4, 0x34, + 0x6b, 0xf4, 0x3a, 0x1b, 0xf8, 0xc4, 0x46, 0xbf, 0x00, 0x97, 0x34, 0xdc, 0xb5, 0xb0, 0xaa, 0x38, + 0x58, 0x93, 0x2d, 0xc5, 0x68, 0x61, 0x59, 0x37, 0xf6, 0x4c, 0xbb, 0x34, 0x73, 0x2d, 0x79, 0x37, + 0xfb, 0xf0, 0xed, 0x08, 0xf9, 0x12, 0x29, 0x55, 0x37, 0xf6, 0xcc, 0xd5, 0xd4, 0x8f, 0x4e, 0x17, + 0xa7, 0xa4, 0x0b, 0xbe, 0x04, 0xef, 0x96, 0x8d, 0x1a, 0x90, 0xe3, 0xd5, 0xb5, 0xb0, 0x62, 0x9b, + 0x46, 0x69, 0xf6, 0x9a, 0x70, 0x37, 0xff, 0x70, 0x29, 0x4a, 0x60, 0x48, 0x35, 0xe4, 0xb2, 0xd7, + 0xc1, 0x12, 0xe5, 0x92, 0xe6, 0xac, 0xc0, 0x15, 0x7a, 0x0b, 0x32, 0xa4, 0x25, 0xbb, 0x27, 0x0e, + 0xb6, 0x4b, 0x69, 0xda, 0x14, 0xd2, 0xb4, 0x55, 0x72, 0x2d, 0x7e, 0x0a, 0x73, 0x41, 0x56, 0x84, + 0x20, 0x2f, 0x55, 0x1b, 0x3b, 0x9b, 0x55, 0x79, 0x67, 0x6b, 0x63, 0xeb, 0xf9, 0xab, 0xad, 0xe2, + 0x14, 0xba, 0x00, 0x45, 0x4e, 0xdb, 0xa8, 0x7e, 0x26, 0x3f, 0xab, 0x6f, 0xd6, 0x9b, 0x45, 0xa1, + 0x9c, 0xfa, 0xb5, 0xdf, 0xb9, 0x3a, 0x25, 0x6e, 0x03, 0x3c, 0xc5, 0x0e, 0x1f, 0x60, 0x68, 0x15, + 0x66, 0xf6, 0x69, 0x7d, 0x4a, 0x02, 0xd5, 0xf4, 0xb5, 0xc8, 0x8a, 0x07, 0x06, 0xe3, 0x6a, 0x9a, + 0x68, 0xe3, 0xc7, 0xa7, 0x8b, 0x82, 0xc4, 0x39, 0xc5, 0x3f, 0x14, 0x20, 0x4b, 0x45, 0xb2, 0xf6, + 0xa1, 0x4a, 0x9f, 0xcc, 0xeb, 0x63, 0x95, 0x31, 0x28, 0x14, 0x2d, 0xc1, 0xf4, 0xa1, 0xd2, 0xee, + 0xe1, 0x52, 0x82, 0xca, 0x28, 0x45, 0xc8, 0x78, 0x49, 0xee, 0x4b, 0xac, 0x18, 0x7a, 0x0c, 0x73, + 0xba, 0xe1, 0x60, 0xc3, 0x91, 0x19, 0x5b, 0x72, 0x0c, 0x5b, 0x96, 0x95, 0xa6, 0x17, 0xe2, 0x3f, + 0x13, 0x00, 0xb6, 0x7b, 0x71, 0x2a, 0x05, 0x7d, 0x73, 0xc2, 0xfa, 0xf3, 0xd1, 0xc5, 0x5b, 0x71, + 0x09, 0x66, 0x74, 0xa3, 0xad, 0x1b, 0xac, 0xfe, 0x69, 0x89, 0x5f, 0xa1, 0x0b, 0x30, 0xbd, 0xdb, + 0xd6, 0x0d, 0x8d, 0xce, 0x87, 0xb4, 0xc4, 0x2e, 0x44, 0x09, 0xb2, 0xb4, 0xd6, 0x31, 0xea, 0x5d, + 0xfc, 0x37, 0x09, 0xb8, 0x58, 0x31, 0x0d, 0x4d, 0x27, 0x53, 0x52, 0x69, 0x7f, 0x2d, 0xb4, 0xb2, + 0x0e, 0x81, 0xc9, 0x27, 0xe3, 0xe3, 0xee, 0x84, 0x7d, 0x8c, 0x7c, 0xae, 0xea, 0x71, 0x97, 0xd2, + 0xa2, 0x35, 0x89, 0xbe, 0x09, 0x97, 0x95, 0x76, 0xdb, 0x3c, 0x92, 0xf5, 0x3d, 0x59, 0x33, 0xb1, + 0x2d, 0x1b, 0xa6, 0x23, 0xe3, 0x63, 0xdd, 0x76, 0xa8, 0x29, 0x49, 0x4b, 0x0b, 0xf4, 0x76, 0x7d, + 0x6f, 0xcd, 0xc4, 0xf6, 0x96, 0xe9, 0x54, 0xc9, 0x2d, 0x32, 0x4f, 0x49, 0x65, 0xd8, 0x3c, 0x9d, + 0x21, 0xe6, 0x57, 0x4a, 0xe3, 0xe3, 0x2e, 0x9b, 0xa7, 0xbf, 0x04, 0x97, 0xfa, 0xf5, 0x18, 0x67, + 0x3f, 0xfd, 0xb1, 0x00, 0xf9, 0xba, 0xa1, 0x3b, 0x5f, 0x8b, 0x0e, 0xf2, 0x94, 0x9a, 0x0c, 0x2a, + 0xf5, 0x3e, 0x14, 0xf7, 0x14, 0xbd, 0xfd, 0xdc, 0x68, 0x9a, 0x9d, 0x5d, 0xdb, 0x31, 0x0d, 0x6c, + 0x73, 0xad, 0x0f, 0xd0, 0xc5, 0x97, 0x50, 0xf0, 0x5a, 0x13, 0xa7, 0x9a, 0x1c, 0x28, 0xd6, 0x0d, + 0xd5, 0xc2, 0x1d, 0x6c, 0xc4, 0xaa, 0xa7, 0xb7, 0x21, 0xa3, 0xbb, 0x72, 0xa9, 0xae, 0x92, 0x92, + 0x4f, 0x10, 0x7b, 0x30, 0x1f, 0x78, 0x6a, 0x9c, 0x66, 0x91, 0x2c, 0x0d, 0xf8, 0x48, 0xf6, 0xfb, + 0x88, 0x2c, 0x0d, 0xf8, 0x88, 0x99, 0xb1, 0x06, 0xe4, 0xd6, 0x70, 0x1b, 0x3b, 0x38, 0x4e, 0xeb, + 0xbe, 0x03, 0x79, 0x57, 0x68, 0x9c, 0x1d, 0xf3, 0x37, 0x04, 0x40, 0x5c, 0x2e, 0x59, 0x4d, 0xe3, + 0xec, 0x9b, 0x45, 0xe2, 0x42, 0x38, 0x3d, 0xcb, 0x60, 0xbe, 0x00, 0x1b, 0x93, 0xc0, 0x48, 0xd4, + 0x1d, 0xf0, 0xad, 0x6c, 0x2a, 0x68, 0x65, 0xb9, 0x1b, 0x73, 0x04, 0x0b, 0xa1, 0x8a, 0xc5, 0xdb, + 0x7d, 0x29, 0x5a, 0xa7, 0xc4, 0xb5, 0x64, 0xd0, 0x57, 0xa3, 0x44, 0xf1, 0x07, 0x02, 0xcc, 0x57, + 0xda, 0x58, 0xb1, 0x62, 0xd7, 0xc8, 0xb7, 0x20, 0xad, 0x61, 0x45, 0xa3, 0x4d, 0x66, 0x13, 0xfb, + 0x9d, 0x80, 0x14, 0xe2, 0xd1, 0x2e, 0xed, 0xb7, 0xd5, 0xa5, 0xa6, 0xeb, 0xeb, 0xf2, 0xd9, 0xed, + 0x31, 0x89, 0x9f, 0x01, 0x0a, 0xd6, 0x2c, 0xce, 0x81, 0xf0, 0xb7, 0x05, 0x40, 0x12, 0x3e, 0xc4, + 0x96, 0x13, 0x7b, 0xb3, 0xd7, 0x20, 0xeb, 0x28, 0x56, 0x0b, 0x3b, 0x32, 0xf1, 0xe2, 0xcf, 0xd3, + 0x72, 0x60, 0x7c, 0x84, 0x2c, 0x7e, 0x0e, 0x0b, 0xa1, 0xfa, 0xc5, 0xd9, 0xf8, 0xff, 0x29, 0x40, + 0xb6, 0xa1, 0x2a, 0x46, 0x9c, 0xad, 0xfe, 0x14, 0xb2, 0xb6, 0xaa, 0x18, 0xf2, 0x9e, 0x69, 0x75, + 0x14, 0x87, 0x0e, 0xf1, 0x7c, 0xa8, 0xd5, 0x9e, 0x07, 0xad, 0x2a, 0xc6, 0x13, 0x5a, 0x48, 0x02, + 0xdb, 0xfb, 0x8d, 0x5e, 0x40, 0xf6, 0x00, 0x9f, 0xc8, 0x1c, 0x8d, 0xd1, 0xf5, 0x2f, 0xff, 0xf0, + 0xfd, 0x00, 0xff, 0xc1, 0xe1, 0x92, 0x0b, 0xe2, 0x96, 0x02, 0x20, 0x6e, 0x89, 0x70, 0x2c, 0x35, + 0x1c, 0x0b, 0x1b, 0x2d, 0x67, 0x5f, 0x82, 0x03, 0x7c, 0xf2, 0x8c, 0xc9, 0x60, 0x13, 0x6b, 0x3d, + 0x95, 0x4e, 0x16, 0x53, 0xe2, 0xff, 0x16, 0x60, 0x8e, 0x35, 0x39, 0xce, 0x89, 0xf5, 0x01, 0xa4, + 0x2c, 0xf3, 0x88, 0x4d, 0xac, 0xec, 0xc3, 0xb7, 0x22, 0x44, 0x6c, 0xe0, 0x93, 0xe0, 0xca, 0x45, + 0x8b, 0xa3, 0x55, 0xe0, 0x7e, 0xa0, 0x4c, 0xb9, 0x93, 0x93, 0x72, 0x03, 0xe3, 0x92, 0x88, 0x8c, + 0x3b, 0x50, 0xd8, 0x55, 0x1c, 0x75, 0x5f, 0xb6, 0x78, 0x25, 0xc9, 0x2a, 0x97, 0xbc, 0x3b, 0x27, + 0xe5, 0x29, 0xd9, 0xad, 0xba, 0x4d, 0x5a, 0xce, 0x46, 0xba, 0x8d, 0xff, 0x82, 0xf5, 0xf9, 0xff, + 0x11, 0xf8, 0x1c, 0x72, 0x5b, 0xfe, 0x17, 0xad, 0xeb, 0x7f, 0x23, 0x01, 0x97, 0x2b, 0xfb, 0x58, + 0x3d, 0xa8, 0x98, 0x86, 0xad, 0xdb, 0x0e, 0xd1, 0x5d, 0x9c, 0xfd, 0xff, 0x16, 0x64, 0x8e, 0x74, + 0x67, 0x5f, 0xd6, 0xf4, 0xbd, 0x3d, 0x6a, 0xe7, 0xd2, 0x52, 0x9a, 0x10, 0xd6, 0xf4, 0xbd, 0x3d, + 0xf4, 0x08, 0x52, 0x1d, 0x53, 0x63, 0xee, 0x72, 0xfe, 0xe1, 0x62, 0x84, 0x78, 0x5a, 0x35, 0xbb, + 0xd7, 0xd9, 0x34, 0x35, 0x2c, 0xd1, 0xc2, 0xe8, 0x2a, 0x80, 0x4a, 0xa8, 0x5d, 0x53, 0x37, 0x1c, + 0xbe, 0x4e, 0x06, 0x28, 0xa8, 0x06, 0x19, 0x07, 0x5b, 0x1d, 0xdd, 0x50, 0x1c, 0x5c, 0x9a, 0xa6, + 0xca, 0xbb, 0x19, 0x59, 0xf1, 0x6e, 0x5b, 0x57, 0x95, 0x35, 0x6c, 0xab, 0x96, 0xde, 0x75, 0x4c, + 0x8b, 0x6b, 0xd1, 0x67, 0x16, 0xbf, 0x97, 0x82, 0xd2, 0xa0, 0x6e, 0xe2, 0x1c, 0x21, 0xdb, 0x30, + 0x43, 0xf0, 0x75, 0xdb, 0xe1, 0x63, 0xe4, 0xe1, 0x30, 0x15, 0x44, 0xd4, 0x80, 0xe2, 0xf4, 0xb6, + 0xc3, 0xab, 0xcd, 0xe5, 0x94, 0xff, 0x95, 0x00, 0x33, 0xec, 0x06, 0x7a, 0x00, 0x69, 0xbe, 0xa1, + 0xa0, 0xd1, 0x3a, 0x26, 0x57, 0x2f, 0x9d, 0x9d, 0x2e, 0xce, 0xb2, 0x3d, 0x82, 0xb5, 0x2f, 0xfd, + 0x9f, 0xd2, 0x2c, 0x2d, 0x57, 0xd7, 0x48, 0x6f, 0xd9, 0x8e, 0x62, 0x39, 0x74, 0xdb, 0x26, 0xc1, + 0x70, 0x03, 0x25, 0x6c, 0xe0, 0x13, 0xb4, 0x0e, 0x33, 0xb6, 0xa3, 0x38, 0x3d, 0x9b, 0xf7, 0xd7, + 0xb9, 0x2a, 0xdb, 0xa0, 0x9c, 0x12, 0x97, 0x40, 0x1c, 0x1d, 0x0d, 0x3b, 0x8a, 0xde, 0xa6, 0x1d, + 0x98, 0x91, 0xf8, 0x95, 0xf8, 0x9b, 0x02, 0xcc, 0xb0, 0xa2, 0xe8, 0x32, 0x2c, 0x48, 0x2b, 0x5b, + 0x4f, 0xab, 0x72, 0x7d, 0x6b, 0xad, 0xda, 0xac, 0x4a, 0x9b, 0xf5, 0xad, 0x95, 0x66, 0xb5, 0x38, + 0x85, 0x2e, 0x01, 0x72, 0x6f, 0x54, 0x9e, 0x6f, 0x35, 0xea, 0x8d, 0x66, 0x75, 0xab, 0x59, 0x14, + 0xe8, 0xde, 0x02, 0xa5, 0x07, 0xa8, 0x09, 0x74, 0x13, 0xae, 0xf5, 0x53, 0xe5, 0x46, 0x73, 0xa5, + 0xd9, 0x90, 0xab, 0x8d, 0x66, 0x7d, 0x73, 0xa5, 0x59, 0x5d, 0x2b, 0x26, 0x47, 0x94, 0x22, 0x0f, + 0x91, 0xa4, 0x6a, 0xa5, 0x59, 0x4c, 0x89, 0x0e, 0x5c, 0x94, 0xb0, 0x6a, 0x76, 0xba, 0x3d, 0x07, + 0x93, 0x5a, 0xda, 0x71, 0xce, 0x94, 0xcb, 0x30, 0xab, 0x59, 0x27, 0xb2, 0xd5, 0x33, 0xf8, 0x3c, + 0x99, 0xd1, 0xac, 0x13, 0xa9, 0x67, 0x88, 0xff, 0x44, 0x80, 0x4b, 0xfd, 0x8f, 0x8d, 0x73, 0x10, + 0xbe, 0x80, 0xac, 0xa2, 0x69, 0x58, 0x93, 0x35, 0xdc, 0x76, 0x14, 0xee, 0x8c, 0xdc, 0x0f, 0x48, + 0xe2, 0x9b, 0x6d, 0x4b, 0xde, 0x66, 0xdb, 0xe6, 0xcb, 0x4a, 0x85, 0x56, 0x64, 0x8d, 0x70, 0xb8, + 0xe6, 0x87, 0x0a, 0xa1, 0x14, 0xf1, 0xbf, 0xa7, 0x20, 0x57, 0x35, 0xb4, 0xe6, 0x71, 0xac, 0x6b, + 0xc9, 0x25, 0x98, 0x51, 0xcd, 0x4e, 0x47, 0x77, 0x5c, 0x05, 0xb1, 0x2b, 0xf4, 0xb3, 0x01, 0x27, + 0x32, 0x39, 0x81, 0x2b, 0xe5, 0xbb, 0x8f, 0xe8, 0x3b, 0x70, 0x99, 0x58, 0x4d, 0xcb, 0x50, 0xda, + 0x32, 0x93, 0x26, 0x3b, 0x96, 0xde, 0x6a, 0x61, 0x8b, 0x6f, 0xf0, 0xdd, 0x8d, 0xa8, 0x67, 0x9d, + 0x73, 0x54, 0x28, 0x43, 0x93, 0x95, 0x97, 0x2e, 0xea, 0x51, 0x64, 0xf4, 0x09, 0x00, 0x59, 0x8a, + 0xe8, 0xa6, 0xa1, 0xcd, 0xed, 0xd1, 0xb0, 0x5d, 0x43, 0xd7, 0x04, 0x11, 0x06, 0x72, 0x6d, 0xa3, + 0x65, 0x82, 0x18, 0x5e, 0xf7, 0x74, 0x0b, 0xcb, 0x0f, 0xba, 0x2a, 0x85, 0xf2, 0xe9, 0xd5, 0xfc, + 0xd9, 0xe9, 0x22, 0x48, 0x8c, 0xfc, 0x60, 0xbb, 0x42, 0x10, 0x04, 0xfb, 0xdd, 0x55, 0xd1, 0x2b, + 0xb8, 0x17, 0xd8, 0x91, 0x20, 0x2b, 0x2f, 0x6f, 0x96, 0xe2, 0xc8, 0xfb, 0x7a, 0x6b, 0x1f, 0x5b, + 0xb2, 0xb7, 0x71, 0x4c, 0x77, 0xf0, 0xd2, 0xd2, 0x4d, 0x9f, 0xa1, 0xa2, 0x18, 0xac, 0xf6, 0x2b, + 0x4e, 0x8d, 0x16, 0xf6, 0x74, 0x46, 0x94, 0xdf, 0x35, 0x75, 0xdb, 0x34, 0x4a, 0x19, 0xa6, 0x7c, + 0x76, 0x85, 0xee, 0x41, 0xd1, 0x39, 0x36, 0xe4, 0x7d, 0xac, 0x58, 0xce, 0x2e, 0x56, 0x1c, 0xb2, + 0x4a, 0x03, 0x2d, 0x51, 0x70, 0x8e, 0x8d, 0x5a, 0x80, 0x8c, 0x5e, 0x40, 0x51, 0x37, 0xe4, 0xbd, + 0xb6, 0xde, 0xda, 0x77, 0xe4, 0x23, 0x4b, 0x77, 0xb0, 0x5d, 0x9a, 0xa7, 0x0a, 0x89, 0x1a, 0xb7, + 0x0d, 0xbe, 0x93, 0xab, 0xbd, 0x22, 0x25, 0xb9, 0x6a, 0xf2, 0xba, 0xf1, 0x84, 0xf2, 0x53, 0xa2, + 0xbd, 0x9e, 0x4a, 0xcf, 0x16, 0xd3, 0xe2, 0x7f, 0x16, 0x20, 0xef, 0x0e, 0xb7, 0x38, 0x67, 0xc6, + 0x5d, 0x28, 0x9a, 0x06, 0x96, 0xbb, 0xfb, 0x8a, 0x8d, 0xb9, 0x1e, 0xf9, 0x82, 0x93, 0x37, 0x0d, + 0xbc, 0x4d, 0xc8, 0x4c, 0x5d, 0x68, 0x1b, 0xe6, 0x6d, 0x47, 0x69, 0xe9, 0x46, 0x2b, 0xa0, 0xde, + 0xe9, 0xc9, 0xdd, 0xfa, 0x22, 0xe7, 0xf6, 0xe8, 0x21, 0x2f, 0xe5, 0x4f, 0x04, 0x98, 0x5f, 0xd1, + 0x3a, 0xba, 0xd1, 0xe8, 0xb6, 0xf5, 0x58, 0x77, 0x0b, 0x6e, 0x42, 0xc6, 0x26, 0x32, 0x7d, 0x83, + 0xef, 0x63, 0xbf, 0x34, 0xbd, 0x43, 0x2c, 0xff, 0x33, 0x28, 0xe0, 0xe3, 0xae, 0xce, 0x5e, 0x06, + 0x30, 0xc8, 0x92, 0x9a, 0xbc, 0x6d, 0x79, 0x9f, 0x97, 0xdc, 0xe2, 0x6d, 0xfa, 0x0c, 0x50, 0xb0, + 0x49, 0x71, 0x62, 0x97, 0xcf, 0x60, 0x81, 0x8a, 0xde, 0x31, 0xec, 0x98, 0xf5, 0x25, 0xfe, 0x22, + 0x5c, 0x08, 0x8b, 0x8e, 0xb3, 0xde, 0xaf, 0x78, 0x2f, 0x6f, 0x62, 0x2b, 0x56, 0xb8, 0xe9, 0xe9, + 0x9a, 0x0b, 0x8e, 0xb3, 0xce, 0xbf, 0x2a, 0xc0, 0x15, 0x2a, 0x9b, 0xbe, 0x2f, 0xd9, 0xc3, 0xd6, + 0x33, 0xac, 0xd8, 0xb1, 0x62, 0xe5, 0x1b, 0x30, 0xc3, 0x30, 0x2f, 0x1d, 0x9f, 0xd3, 0xab, 0x59, + 0xe2, 0xb9, 0x34, 0x1c, 0xd3, 0x22, 0x9e, 0x0b, 0xbf, 0x25, 0x2a, 0x50, 0x8e, 0xaa, 0x45, 0xcc, + 0xdb, 0x01, 0xf3, 0xdc, 0x69, 0x24, 0x43, 0xb9, 0xb2, 0x4f, 0x7c, 0x26, 0x54, 0x85, 0xac, 0x4a, + 0x7f, 0xc9, 0xce, 0x49, 0x17, 0x53, 0xf9, 0xf9, 0x51, 0xfe, 0x26, 0x63, 0x6b, 0x9e, 0x74, 0x31, + 0x71, 0x5a, 0xdd, 0xdf, 0x44, 0x51, 0x81, 0x46, 0x8e, 0xf4, 0x58, 0xe9, 0x3c, 0xa2, 0x65, 0x5d, + 0xd7, 0x8f, 0xeb, 0xe0, 0x9f, 0x26, 0xb9, 0x12, 0xd8, 0x33, 0x78, 0xf1, 0x58, 0x7d, 0x94, 0xcf, + 0x43, 0xaf, 0xab, 0x82, 0x0d, 0x4f, 0x9c, 0xa3, 0xe1, 0x81, 0x3d, 0x73, 0x9f, 0x8a, 0x3e, 0x83, + 0xc0, 0xae, 0xb8, 0xcc, 0xda, 0xe4, 0xa2, 0x9f, 0xf3, 0xa8, 0x63, 0xde, 0x97, 0xc2, 0xe8, 0x36, + 0xaa, 0x40, 0x1a, 0x1f, 0x77, 0x65, 0x0d, 0xdb, 0x2a, 0x37, 0x5c, 0xe2, 0xb0, 0xf7, 0x6a, 0x03, + 0x78, 0x60, 0x16, 0x1f, 0x77, 0x09, 0x11, 0xed, 0x90, 0xd5, 0xcb, 0x75, 0x15, 0x68, 0xb5, 0xed, + 0xf1, 0xf0, 0xc2, 0x1f, 0x29, 0x5c, 0x5c, 0xc1, 0xf3, 0x12, 0x98, 0x08, 0xf1, 0x87, 0x02, 0xbc, + 0x15, 0xd9, 0x6b, 0x71, 0x2e, 0x64, 0x9f, 0x40, 0x8a, 0x36, 0x3e, 0x71, 0xce, 0xc6, 0x53, 0x2e, + 0xf1, 0x77, 0xdd, 0x39, 0x2e, 0xe1, 0xb6, 0x49, 0x14, 0xfb, 0x15, 0xec, 0x87, 0xcd, 0xba, 0x1d, + 0x9e, 0x38, 0x77, 0x87, 0xbb, 0xac, 0x9e, 0x11, 0xe8, 0xab, 0x66, 0x9c, 0x46, 0xe0, 0xfb, 0x02, + 0x2c, 0x78, 0x3e, 0x4d, 0xcc, 0xee, 0xed, 0x07, 0x90, 0x34, 0xcc, 0xa3, 0xf3, 0x6c, 0x06, 0x92, + 0xf2, 0x64, 0x49, 0x0a, 0xd7, 0x28, 0xce, 0xf6, 0xfe, 0xeb, 0x04, 0x64, 0x9e, 0x56, 0xe2, 0x6c, + 0xe5, 0x27, 0x7c, 0xa3, 0x99, 0x4d, 0xec, 0xa8, 0xa1, 0xe8, 0x3d, 0x6f, 0xe9, 0x69, 0x65, 0x03, + 0x9f, 0xb8, 0x43, 0x91, 0x70, 0xa1, 0x15, 0xc8, 0x38, 0xfb, 0x16, 0xb6, 0xf7, 0xcd, 0xb6, 0x76, + 0x1e, 0x1f, 0xc4, 0xe7, 0x2a, 0x63, 0x98, 0xa6, 0x72, 0xdd, 0xe0, 0x05, 0x21, 0x22, 0x78, 0x81, + 0x3c, 0xc6, 0x73, 0xe3, 0x12, 0xe7, 0x79, 0x4c, 0xc0, 0x7f, 0x9b, 0x2e, 0xce, 0x88, 0x2f, 0x00, + 0x48, 0x73, 0xe2, 0xec, 0x92, 0xbf, 0x92, 0x84, 0xfc, 0x76, 0xcf, 0xde, 0x8f, 0x79, 0xf4, 0x55, + 0x00, 0xba, 0x3d, 0x9b, 0xe2, 0x83, 0x63, 0x83, 0xb7, 0x79, 0x4c, 0x5c, 0x84, 0xdb, 0x68, 0xc6, + 0xd7, 0x3c, 0x36, 0x50, 0x8d, 0x0b, 0xc1, 0xb2, 0x1f, 0x5c, 0x71, 0x63, 0x14, 0x92, 0x6c, 0x1e, + 0x1b, 0x9b, 0xd8, 0x83, 0x90, 0x4c, 0x12, 0x26, 0x92, 0x3e, 0x81, 0x59, 0x72, 0x21, 0x3b, 0xe6, + 0x79, 0xba, 0x79, 0x86, 0xf0, 0x34, 0x4d, 0xf4, 0x18, 0x32, 0x8c, 0x9b, 0x2c, 0x4d, 0x33, 0x74, + 0x69, 0x8a, 0x6a, 0x0b, 0x57, 0x23, 0x5d, 0x94, 0xd2, 0x94, 0x95, 0x2c, 0x44, 0x17, 0x60, 0x7a, + 0xcf, 0xb4, 0x54, 0x4c, 0x23, 0x26, 0xd2, 0x12, 0xbb, 0x60, 0xfd, 0xb9, 0x9e, 0x4a, 0xa7, 0x8b, + 0x99, 0xf5, 0x54, 0x3a, 0x53, 0x04, 0xf1, 0x37, 0x05, 0x28, 0x78, 0x1d, 0x11, 0xa7, 0xb5, 0xae, + 0x84, 0xb4, 0x78, 0xfe, 0xae, 0x20, 0x0a, 0x14, 0xff, 0x2d, 0x75, 0x57, 0x54, 0xf3, 0x90, 0xf6, + 0x4c, 0x9c, 0x23, 0xe5, 0x31, 0x0b, 0x9d, 0x49, 0x9c, 0xb7, 0x77, 0x69, 0x14, 0xcd, 0x03, 0xb8, + 0xa0, 0x77, 0x88, 0x1d, 0xd7, 0x9d, 0xf6, 0x09, 0xc7, 0x54, 0x0e, 0x76, 0xdf, 0xcf, 0x2e, 0xf8, + 0xf7, 0x2a, 0xee, 0x2d, 0xf1, 0xef, 0xd2, 0xdd, 0x69, 0xbf, 0x25, 0x71, 0xaa, 0xba, 0x0e, 0x39, + 0x8b, 0x89, 0x26, 0x3e, 0xc7, 0x39, 0xb5, 0x3d, 0xe7, 0xb1, 0x12, 0x85, 0xff, 0x76, 0x02, 0x0a, + 0x2f, 0x7a, 0xd8, 0x3a, 0xf9, 0x3a, 0xa9, 0xfb, 0x36, 0x14, 0x8e, 0x14, 0xdd, 0x91, 0xf7, 0x4c, + 0x4b, 0xee, 0x75, 0x35, 0xc5, 0x71, 0xe3, 0x37, 0x72, 0x84, 0xfc, 0xc4, 0xb4, 0x76, 0x28, 0x11, + 0x61, 0x40, 0x07, 0x86, 0x79, 0x64, 0xc8, 0x84, 0x4c, 0x51, 0xec, 0xb1, 0xc1, 0xb7, 0x8c, 0x57, + 0x3f, 0xfc, 0x4f, 0xa7, 0x8b, 0x8f, 0x26, 0x8a, 0xca, 0xa2, 0x11, 0x68, 0xbd, 0x9e, 0xae, 0x2d, + 0xed, 0xec, 0xd4, 0xd7, 0xa4, 0x22, 0x15, 0xf9, 0x8a, 0x49, 0x6c, 0x1e, 0x1b, 0xb6, 0xf8, 0xf7, + 0x12, 0x50, 0xf4, 0x75, 0x14, 0x67, 0x47, 0x56, 0x21, 0xfb, 0xba, 0x87, 0x2d, 0xfd, 0x0d, 0xba, + 0x11, 0x38, 0x23, 0x31, 0x3b, 0x9f, 0xc3, 0x5c, 0x48, 0x03, 0xc9, 0x9f, 0x4e, 0x03, 0xd9, 0x23, + 0xbf, 0xf1, 0xe8, 0x3e, 0xcc, 0x3b, 0xc7, 0x86, 0xcc, 0xe2, 0xf1, 0x58, 0x0c, 0x87, 0x1b, 0x76, + 0x50, 0x70, 0x88, 0x3e, 0x08, 0x9d, 0xc6, 0x6f, 0xd8, 0xe2, 0x1f, 0x0a, 0x80, 0xa8, 0xa2, 0xea, + 0x6c, 0x4f, 0xff, 0xeb, 0x32, 0x9e, 0xee, 0x42, 0x91, 0x46, 0x38, 0xca, 0xfa, 0x9e, 0xdc, 0xd1, + 0x6d, 0x5b, 0x37, 0x5a, 0x7c, 0x40, 0xe5, 0x29, 0xbd, 0xbe, 0xb7, 0xc9, 0xa8, 0xe2, 0xff, 0x0f, + 0x0b, 0xa1, 0x06, 0xc4, 0xd9, 0xd9, 0xd7, 0x61, 0x6e, 0xcf, 0xec, 0x19, 0x9a, 0xcc, 0xde, 0x78, + 0xf0, 0xed, 0xc0, 0x2c, 0xa5, 0xb1, 0xe7, 0x89, 0xff, 0x2d, 0x01, 0x17, 0x24, 0x6c, 0x9b, 0xed, + 0x43, 0x1c, 0xbf, 0x0a, 0x6b, 0xc0, 0xdf, 0xb5, 0xc8, 0x6f, 0xa4, 0xc9, 0x0c, 0x63, 0x66, 0xcb, + 0x5c, 0x78, 0x4f, 0xfd, 0xe6, 0xe8, 0x11, 0x3b, 0xb8, 0x8b, 0xce, 0xf7, 0xe4, 0x52, 0xa1, 0x3d, + 0x39, 0x13, 0x0a, 0x7a, 0xcb, 0x30, 0x89, 0x4d, 0xb3, 0xf1, 0x6b, 0xa3, 0xd7, 0x71, 0x91, 0xca, + 0xd2, 0xa8, 0x4a, 0xd6, 0x19, 0x4b, 0x03, 0xbf, 0xde, 0xea, 0x75, 0xa8, 0xef, 0xbc, 0x7a, 0x89, + 0xd4, 0xf7, 0xec, 0x74, 0x31, 0x1f, 0xba, 0x67, 0x4b, 0x79, 0xdd, 0xbb, 0x26, 0xd2, 0xc5, 0x6f, + 0xc3, 0xc5, 0x3e, 0x65, 0xc7, 0xe9, 0xf1, 0xfc, 0xcb, 0x24, 0x5c, 0x09, 0x8b, 0x8f, 0x1b, 0x7f, + 0x7c, 0xdd, 0x3b, 0xb4, 0x06, 0xb9, 0x8e, 0x6e, 0xbc, 0xd9, 0xd6, 0xe2, 0x5c, 0x47, 0x37, 0xfc, + 0x6d, 0xdc, 0x88, 0xa1, 0x31, 0xf3, 0x95, 0x0e, 0x0d, 0x05, 0xca, 0x51, 0x7d, 0x17, 0xe7, 0xf8, + 0xf8, 0x35, 0x01, 0xe6, 0xe2, 0xde, 0x33, 0x7b, 0xb3, 0x78, 0x33, 0xb1, 0x09, 0xb9, 0xaf, 0x60, + 0x93, 0xed, 0xb7, 0x05, 0x40, 0x4d, 0xab, 0x67, 0x10, 0x50, 0xfb, 0xcc, 0x6c, 0xc5, 0xd9, 0xcc, + 0x0b, 0x30, 0xad, 0x1b, 0x1a, 0x3e, 0xa6, 0xcd, 0x4c, 0x49, 0xec, 0x22, 0xf4, 0xea, 0x30, 0x39, + 0xd1, 0xab, 0x43, 0xf1, 0x73, 0x58, 0x08, 0x55, 0x31, 0xce, 0xf6, 0xff, 0xfd, 0x04, 0x2c, 0xf0, + 0x86, 0xc4, 0xbe, 0xbd, 0xf8, 0x4d, 0x98, 0x6e, 0x13, 0x99, 0x23, 0xfa, 0x99, 0x3e, 0xd3, 0xed, + 0x67, 0x5a, 0x18, 0xfd, 0x1c, 0x40, 0xd7, 0xc2, 0x87, 0x32, 0x63, 0x4d, 0x4e, 0xc4, 0x9a, 0x21, + 0x1c, 0x94, 0x80, 0x7e, 0x01, 0x0a, 0x64, 0x3e, 0x77, 0x2d, 0xb3, 0x6b, 0xda, 0xc4, 0x65, 0xb1, + 0x27, 0x43, 0x39, 0xf3, 0x67, 0xa7, 0x8b, 0xb9, 0x4d, 0xdd, 0xd8, 0xe6, 0x8c, 0xcd, 0x86, 0x44, + 0x0c, 0x83, 0x77, 0x69, 0x8b, 0xff, 0x41, 0x80, 0x0b, 0x5f, 0xd9, 0x56, 0xec, 0xff, 0x0b, 0x5d, + 0x89, 0x2f, 0xa1, 0x48, 0x7f, 0xd4, 0x8d, 0x3d, 0x33, 0xce, 0x4d, 0xf1, 0xef, 0x09, 0x30, 0x1f, + 0x10, 0x1c, 0xa7, 0x7f, 0xf2, 0x46, 0x7a, 0x12, 0x7f, 0x91, 0x78, 0x2c, 0xc1, 0x41, 0x1e, 0xe7, + 0x14, 0xfa, 0xfd, 0x04, 0x5c, 0xaa, 0xb0, 0x57, 0xc8, 0x6e, 0x4c, 0x45, 0x9c, 0x23, 0xa3, 0x04, + 0xb3, 0x87, 0xd8, 0xb2, 0x75, 0x93, 0xad, 0x9e, 0x39, 0xc9, 0xbd, 0x44, 0x65, 0x48, 0xdb, 0x86, + 0xd2, 0xb5, 0xf7, 0x4d, 0xf7, 0xdd, 0x99, 0x77, 0xed, 0xc5, 0x7f, 0x4c, 0xbf, 0x79, 0xfc, 0xc7, + 0xcc, 0xe8, 0xf8, 0x8f, 0xd9, 0x9f, 0x22, 0xfe, 0x83, 0xbf, 0xa8, 0xfa, 0x77, 0x02, 0x5c, 0x1e, + 0xd0, 0x5c, 0x9c, 0xa3, 0xe5, 0xbb, 0x90, 0x55, 0xb9, 0x60, 0x62, 0x6f, 0xd9, 0x5b, 0xb8, 0x3a, + 0x29, 0xf6, 0x86, 0xb0, 0xe3, 0xec, 0x74, 0x11, 0xdc, 0xaa, 0xd6, 0xd7, 0xb8, 0x72, 0xc8, 0x6f, + 0x4d, 0xfc, 0x95, 0x1c, 0x14, 0xaa, 0xc7, 0x6c, 0x07, 0xba, 0xc1, 0x56, 0x79, 0xf4, 0x04, 0xd2, + 0x5d, 0xcb, 0x3c, 0xd4, 0xdd, 0x66, 0xe4, 0x43, 0x2f, 0xff, 0xdd, 0x66, 0xf4, 0x71, 0x6d, 0x73, + 0x0e, 0xc9, 0xe3, 0x45, 0x4d, 0xc8, 0x3c, 0x33, 0x55, 0xa5, 0xfd, 0x44, 0x6f, 0xbb, 0x23, 0xff, + 0xfd, 0xf1, 0x82, 0x96, 0x3c, 0x9e, 0x6d, 0xc5, 0xd9, 0x77, 0x3b, 0xc1, 0x23, 0xa2, 0x3a, 0xa4, + 0x6b, 0x8e, 0xd3, 0x25, 0x37, 0xb9, 0xed, 0xb8, 0x33, 0x81, 0x50, 0xc2, 0xe2, 0xc6, 0x8a, 0xba, + 0xec, 0xa8, 0x09, 0xf3, 0x4f, 0xe9, 0x09, 0xa7, 0x4a, 0xdb, 0xec, 0x69, 0x15, 0xd3, 0xd8, 0xd3, + 0x5b, 0xdc, 0xee, 0xde, 0x9e, 0x40, 0xe6, 0xd3, 0x4a, 0x43, 0x1a, 0x14, 0x80, 0x56, 0x20, 0xdd, + 0x78, 0xc4, 0x85, 0x31, 0xb7, 0xec, 0xd6, 0x04, 0xc2, 0x1a, 0x8f, 0x24, 0x8f, 0x0d, 0xad, 0x43, + 0x76, 0xe5, 0x8b, 0x9e, 0x85, 0xb9, 0x94, 0x99, 0xa1, 0x91, 0x07, 0xfd, 0x52, 0x28, 0x97, 0x14, + 0x64, 0x46, 0x0d, 0xc8, 0xbf, 0x32, 0xad, 0x83, 0xb6, 0xa9, 0xb8, 0x2d, 0x9c, 0xa5, 0xe2, 0xbe, + 0x31, 0x81, 0x38, 0x97, 0x51, 0xea, 0x13, 0x81, 0xbe, 0x0d, 0x05, 0xd2, 0x19, 0x4d, 0x65, 0xb7, + 0xed, 0x56, 0x32, 0x4d, 0xa5, 0xbe, 0x3b, 0x81, 0x54, 0x8f, 0xd3, 0x7d, 0x05, 0xd2, 0x27, 0xaa, + 0x2c, 0x41, 0x2e, 0x34, 0x08, 0x10, 0x82, 0x54, 0x97, 0xf4, 0xb7, 0x40, 0x63, 0x83, 0xe8, 0x6f, + 0xf4, 0x1e, 0xcc, 0x1a, 0xa6, 0x86, 0xdd, 0x19, 0x92, 0x5b, 0xbd, 0x70, 0x76, 0xba, 0x38, 0xb3, + 0x65, 0x6a, 0xcc, 0x21, 0xe1, 0xbf, 0xa4, 0x19, 0x52, 0xa8, 0xae, 0x95, 0xaf, 0x41, 0x8a, 0xf4, + 0x3b, 0x31, 0x4c, 0xbb, 0x8a, 0x8d, 0x77, 0x2c, 0x9d, 0x4b, 0x73, 0x2f, 0xcb, 0xff, 0x38, 0x01, + 0x89, 0xc6, 0x23, 0xe2, 0x72, 0xef, 0xf6, 0xd4, 0x03, 0xec, 0xf0, 0xfb, 0xfc, 0x8a, 0xba, 0xe2, + 0x16, 0xde, 0xd3, 0x99, 0x67, 0x94, 0x91, 0xf8, 0x15, 0x7a, 0x07, 0x40, 0x51, 0x55, 0x6c, 0xdb, + 0xb2, 0x7b, 0xf2, 0x2d, 0x23, 0x65, 0x18, 0x65, 0x03, 0x9f, 0x10, 0x36, 0x1b, 0xab, 0x16, 0x76, + 0xdc, 0xc0, 0x26, 0x76, 0x45, 0xd8, 0x1c, 0xdc, 0xe9, 0xca, 0x8e, 0x79, 0x80, 0x0d, 0x3a, 0x4e, + 0x32, 0xc4, 0xd4, 0x74, 0xba, 0x4d, 0x42, 0x20, 0x56, 0x12, 0x1b, 0x9a, 0x6f, 0xd2, 0x32, 0x92, + 0x77, 0x4d, 0x44, 0x5a, 0xb8, 0xa5, 0xf3, 0x23, 0x5c, 0x19, 0x89, 0x5f, 0x11, 0x2d, 0x29, 0x3d, + 0x67, 0x9f, 0xf6, 0x44, 0x46, 0xa2, 0xbf, 0xd1, 0x6d, 0x28, 0xb0, 0x58, 0x48, 0x19, 0x1b, 0xaa, + 0x4c, 0x8d, 0x6b, 0x86, 0xde, 0xce, 0x31, 0x72, 0xd5, 0x50, 0x89, 0x29, 0x45, 0x8f, 0x80, 0x13, + 0xe4, 0x83, 0x8e, 0x4d, 0x74, 0x0a, 0xa4, 0xd4, 0x6a, 0xe1, 0xec, 0x74, 0x31, 0xdb, 0xa0, 0x37, + 0x36, 0x36, 0x1b, 0xf5, 0x35, 0x29, 0xcb, 0x4a, 0x6d, 0x74, 0xec, 0xba, 0x56, 0xfe, 0x75, 0x01, + 0x92, 0x4f, 0x2b, 0x8d, 0x73, 0xab, 0xcc, 0xad, 0x68, 0x32, 0x50, 0xd1, 0x3b, 0x50, 0xd8, 0xd5, + 0xdb, 0x6d, 0xdd, 0x68, 0x11, 0x2f, 0xe8, 0xbb, 0x58, 0x75, 0x15, 0x96, 0xe7, 0xe4, 0x6d, 0x46, + 0x45, 0xd7, 0x20, 0xab, 0x5a, 0x58, 0xc3, 0x86, 0xa3, 0x2b, 0x6d, 0x9b, 0x6b, 0x2e, 0x48, 0x2a, + 0xff, 0xb2, 0x00, 0xd3, 0x74, 0x06, 0xa0, 0xb7, 0x21, 0xa3, 0x9a, 0x86, 0xa3, 0xe8, 0x06, 0x37, + 0x65, 0x19, 0xc9, 0x27, 0x0c, 0xad, 0xde, 0x75, 0x98, 0x53, 0x54, 0xd5, 0xec, 0x19, 0x8e, 0x6c, + 0x28, 0x1d, 0xcc, 0xab, 0x99, 0xe5, 0xb4, 0x2d, 0xa5, 0x83, 0xd1, 0x22, 0xb8, 0x97, 0xde, 0x81, + 0xc6, 0x8c, 0x04, 0x9c, 0xb4, 0x81, 0x4f, 0xca, 0x7f, 0x24, 0x40, 0xda, 0x9d, 0x33, 0xa4, 0x1a, + 0x2d, 0x6c, 0x60, 0x4b, 0x71, 0x4c, 0xaf, 0x1a, 0x1e, 0xa1, 0x7f, 0xa9, 0xcc, 0xf8, 0x4b, 0xe5, + 0x05, 0x98, 0x76, 0xc8, 0xb4, 0xe0, 0x35, 0x60, 0x17, 0x74, 0x3b, 0xba, 0xad, 0xb4, 0xd8, 0x6e, + 0x5c, 0x46, 0x62, 0x17, 0xa4, 0x31, 0x3c, 0xa4, 0x96, 0x69, 0x84, 0x5f, 0x91, 0x9a, 0xb2, 0xc0, + 0xcf, 0x5d, 0xdc, 0xd2, 0x0d, 0x3a, 0x96, 0x92, 0x12, 0x50, 0xd2, 0x2a, 0xa1, 0xa0, 0xb7, 0x20, + 0xc3, 0x0a, 0x60, 0x43, 0xa3, 0x03, 0x2a, 0x29, 0xa5, 0x29, 0xa1, 0x6a, 0x68, 0x65, 0x0c, 0x19, + 0x6f, 0x72, 0x92, 0x6e, 0xeb, 0xd9, 0x9e, 0x22, 0xe9, 0x6f, 0xf4, 0x3e, 0x5c, 0x78, 0xdd, 0x53, + 0xda, 0xfa, 0x1e, 0xdd, 0x68, 0x23, 0xc5, 0x98, 0xce, 0x58, 0x4b, 0x90, 0x77, 0x8f, 0x4a, 0xa0, + 0xaa, 0x73, 0xe7, 0x72, 0xd2, 0x9f, 0xcb, 0xe2, 0xef, 0x09, 0x30, 0xcf, 0xa2, 0x7a, 0x58, 0x30, + 0x6a, 0x7c, 0x7e, 0xc8, 0xc7, 0x90, 0xd1, 0x14, 0x47, 0x61, 0x47, 0x34, 0x13, 0x23, 0x8f, 0x68, + 0x7a, 0x47, 0x09, 0x14, 0x47, 0xa1, 0xc7, 0x34, 0x11, 0xa4, 0xc8, 0x6f, 0x76, 0x9a, 0x55, 0xa2, + 0xbf, 0xc5, 0xcf, 0x00, 0x05, 0x2b, 0x1a, 0xa7, 0x47, 0x76, 0x0f, 0x2e, 0x12, 0x5d, 0x57, 0x0d, + 0xd5, 0x3a, 0xe9, 0x3a, 0xba, 0x69, 0x3c, 0xa7, 0x7f, 0x6d, 0x54, 0x0c, 0xbc, 0x97, 0xa2, 0xaf, + 0xa3, 0xc4, 0x3f, 0x98, 0x81, 0x5c, 0xf5, 0xb8, 0x6b, 0x5a, 0xb1, 0xee, 0x62, 0xad, 0xc2, 0x2c, + 0x07, 0xfa, 0x23, 0xde, 0x0b, 0xf7, 0x19, 0x73, 0xf7, 0x95, 0x2b, 0x67, 0x44, 0xab, 0x00, 0x2c, + 0x60, 0x94, 0x06, 0x05, 0x25, 0xcf, 0xf1, 0xa6, 0x8c, 0xb2, 0x11, 0x2a, 0xda, 0x82, 0x6c, 0xe7, + 0x50, 0x55, 0xe5, 0x3d, 0xbd, 0xed, 0xf0, 0xb8, 0xbb, 0xe8, 0x10, 0xf1, 0xcd, 0x97, 0x95, 0xca, + 0x13, 0x5a, 0x88, 0x85, 0xc0, 0xf9, 0xd7, 0x12, 0x10, 0x09, 0xec, 0x37, 0x7a, 0x17, 0xf8, 0x91, + 0x1a, 0xd9, 0x76, 0x4f, 0xc9, 0xad, 0xe6, 0xce, 0x4e, 0x17, 0x33, 0x12, 0xa5, 0x36, 0x1a, 0x4d, + 0x29, 0xc3, 0x0a, 0x34, 0x6c, 0x07, 0xdd, 0x80, 0x9c, 0xd9, 0xd1, 0x1d, 0xd9, 0x75, 0x92, 0xb8, + 0x47, 0x39, 0x47, 0x88, 0xae, 0x13, 0x85, 0x9a, 0x70, 0x07, 0x1b, 0x74, 0xb4, 0x93, 0x76, 0xca, + 0xbb, 0x6c, 0xf3, 0xd1, 0x61, 0x33, 0x5a, 0x36, 0xbb, 0x8e, 0xde, 0xd1, 0xbf, 0xa0, 0x6f, 0xa6, + 0xf9, 0x4b, 0xa3, 0x1b, 0xac, 0x38, 0x69, 0xdf, 0x2a, 0xdd, 0x95, 0xe4, 0x65, 0x9f, 0x07, 0x8a, + 0xa2, 0xbf, 0x2a, 0xc0, 0x25, 0xae, 0x48, 0x79, 0x97, 0xc6, 0xb8, 0x2b, 0x6d, 0xdd, 0x39, 0x91, + 0x0f, 0x0e, 0x4b, 0x69, 0xea, 0xb7, 0xfe, 0x6c, 0x64, 0x87, 0x04, 0xc6, 0xc1, 0x92, 0xdb, 0x2d, + 0x27, 0xcf, 0x38, 0xf3, 0xc6, 0x61, 0xd5, 0x70, 0xac, 0x93, 0xd5, 0xcb, 0x67, 0xa7, 0x8b, 0x0b, + 0x83, 0x77, 0x5f, 0x4a, 0x0b, 0xf6, 0x20, 0x0b, 0xaa, 0x01, 0x60, 0x6f, 0x1c, 0xd2, 0x15, 0x23, + 0xda, 0xff, 0x88, 0x1c, 0xb0, 0x52, 0x80, 0x17, 0xdd, 0x85, 0x22, 0x3f, 0xd9, 0xb2, 0xa7, 0xb7, + 0xb1, 0x6c, 0xeb, 0x5f, 0x60, 0xba, 0xb6, 0x24, 0xa5, 0x3c, 0xa3, 0x13, 0x11, 0x0d, 0xfd, 0x0b, + 0x5c, 0xfe, 0x2e, 0x94, 0x86, 0xd5, 0x3e, 0x38, 0x05, 0x32, 0xec, 0x8d, 0xec, 0x47, 0xe1, 0xed, + 0x98, 0x09, 0x86, 0x2a, 0xdf, 0x92, 0xf9, 0x38, 0xf1, 0x91, 0x20, 0xfe, 0x83, 0x04, 0xe4, 0x56, + 0x7b, 0xed, 0x83, 0xe7, 0xdd, 0x46, 0xaf, 0xd3, 0x51, 0xac, 0x13, 0x62, 0x06, 0x99, 0xa1, 0x20, + 0x15, 0x14, 0x98, 0x19, 0xa4, 0x96, 0x40, 0xff, 0x02, 0x93, 0xc5, 0x29, 0x78, 0x2a, 0x9b, 0xc5, + 0xf0, 0xd3, 0x36, 0x04, 0x8e, 0x5a, 0x9b, 0x47, 0x36, 0xfa, 0x08, 0x4a, 0x81, 0x82, 0x74, 0xef, + 0x44, 0xc6, 0x86, 0x63, 0xe9, 0x98, 0xed, 0xff, 0x25, 0xa5, 0x40, 0xbc, 0x4c, 0x9d, 0xdc, 0xae, + 0xb2, 0xbb, 0xa8, 0x09, 0x73, 0xa4, 0xe0, 0x89, 0x4c, 0x97, 0x10, 0x77, 0x7f, 0xf6, 0x41, 0x44, + 0xb3, 0x42, 0xf5, 0x5e, 0xa2, 0xfa, 0xa9, 0x50, 0x1e, 0xfa, 0x53, 0xca, 0x62, 0x9f, 0x52, 0xfe, + 0x14, 0x8a, 0xfd, 0x05, 0x82, 0xba, 0x4c, 0x31, 0x5d, 0x5e, 0x08, 0xea, 0x32, 0x19, 0xd0, 0xd3, + 0x7a, 0x2a, 0x9d, 0x2a, 0x4e, 0x8b, 0x7f, 0x9e, 0x84, 0xbc, 0x3b, 0xcc, 0xe2, 0x04, 0x3a, 0xab, + 0x30, 0x4d, 0x06, 0x85, 0x1b, 0xe3, 0x71, 0x7b, 0xc4, 0xe8, 0xe6, 0x51, 0xe3, 0x64, 0xb0, 0xb8, + 0x20, 0x99, 0xb2, 0xc6, 0x61, 0x70, 0xca, 0xbf, 0x9c, 0x80, 0x14, 0xc5, 0x16, 0x0f, 0x20, 0x45, + 0x17, 0x0a, 0x61, 0x92, 0x85, 0x82, 0x16, 0xf5, 0x96, 0xb3, 0x44, 0xc0, 0x35, 0x25, 0x3e, 0xdf, + 0xbe, 0xf2, 0xc1, 0x83, 0x87, 0xd4, 0xd8, 0xcc, 0x49, 0xfc, 0x0a, 0xad, 0xd2, 0xb0, 0x23, 0xd3, + 0x72, 0xb0, 0xc6, 0x7d, 0xfa, 0x6b, 0xe3, 0xfa, 0xd7, 0x5d, 0x94, 0x5c, 0x3e, 0x74, 0x05, 0x92, + 0xc4, 0x8a, 0xcd, 0xb2, 0x20, 0x85, 0xb3, 0xd3, 0xc5, 0x24, 0xb1, 0x5f, 0x84, 0x86, 0x96, 0x21, + 0x1b, 0x36, 0x19, 0xc4, 0x83, 0xa3, 0x86, 0x31, 0x30, 0xdd, 0xa1, 0xed, 0x4d, 0x2d, 0x86, 0x67, + 0x79, 0x1f, 0xff, 0x8f, 0x14, 0xe4, 0xea, 0x9d, 0xb8, 0x97, 0x94, 0x95, 0x70, 0x0f, 0x47, 0x01, + 0xa1, 0xd0, 0x43, 0x23, 0x3a, 0x38, 0xb4, 0x82, 0x27, 0xcf, 0xb7, 0x82, 0xd7, 0x89, 0xa7, 0xcc, + 0xd3, 0x2c, 0x24, 0x87, 0x60, 0x9e, 0xf0, 0xf3, 0xa9, 0x9f, 0x22, 0x11, 0x1e, 0xff, 0x1c, 0x05, + 0x0d, 0x34, 0xf9, 0x94, 0x3a, 0xe4, 0x6c, 0x94, 0xcd, 0x4c, 0x3e, 0xca, 0x66, 0xb1, 0xa1, 0xd1, + 0x45, 0x2d, 0x6c, 0x51, 0x67, 0xdf, 0xdc, 0xa2, 0x96, 0x1d, 0x3e, 0x58, 0x3f, 0x86, 0xa4, 0xa6, + 0xbb, 0x9d, 0x33, 0xf9, 0x52, 0x4d, 0x98, 0xc6, 0x8c, 0xda, 0x54, 0x70, 0xd4, 0xb2, 0x51, 0x52, + 0xae, 0x03, 0xf8, 0xba, 0x41, 0xd7, 0x60, 0xc6, 0x6c, 0x6b, 0xee, 0x41, 0x92, 0xdc, 0x6a, 0xe6, + 0xec, 0x74, 0x71, 0xfa, 0x79, 0x5b, 0xab, 0xaf, 0x49, 0xd3, 0x66, 0x5b, 0xab, 0x6b, 0x34, 0xc7, + 0x05, 0x3e, 0x92, 0xbd, 0x28, 0xb3, 0x39, 0x69, 0xd6, 0xc0, 0x47, 0x6b, 0xd8, 0x56, 0xf9, 0x80, + 0xfb, 0x2d, 0x01, 0xf2, 0xae, 0xee, 0xe3, 0x35, 0x2a, 0x69, 0xbd, 0xc3, 0x27, 0x59, 0xf2, 0x7c, + 0x93, 0xcc, 0xe5, 0xe3, 0xc7, 0x6b, 0x7f, 0x55, 0xe0, 0x71, 0xc3, 0x0d, 0x55, 0x71, 0x88, 0x53, + 0x11, 0xe3, 0xc4, 0xb8, 0x07, 0x45, 0x4b, 0x31, 0x34, 0xb3, 0xa3, 0x7f, 0x81, 0xd9, 0x46, 0xa8, + 0xcd, 0xdf, 0x5a, 0x16, 0x3c, 0x3a, 0xdd, 0xf5, 0xb3, 0xc5, 0x3f, 0x4a, 0xf0, 0x18, 0x63, 0xaf, + 0x1a, 0x71, 0xaa, 0xeb, 0x3b, 0x30, 0xdf, 0x9f, 0x70, 0xc4, 0x9d, 0xad, 0xef, 0x45, 0xc8, 0x8b, + 0xaa, 0x08, 0x8b, 0x15, 0x74, 0x03, 0xd7, 0xfb, 0x92, 0x8f, 0xd8, 0xa8, 0x02, 0xd9, 0x60, 0x1e, + 0x93, 0xe4, 0xc4, 0x79, 0x4c, 0xc0, 0xf2, 0xb2, 0x97, 0x94, 0x7f, 0x1e, 0xa6, 0xe9, 0xed, 0x37, + 0x30, 0xd1, 0xbc, 0x37, 0xff, 0x2c, 0x01, 0x37, 0x69, 0xed, 0x5f, 0x62, 0x4b, 0xdf, 0x3b, 0xd9, + 0xb6, 0x4c, 0x07, 0xab, 0x0e, 0xd6, 0xfc, 0x53, 0x20, 0xb1, 0xda, 0xbd, 0x4c, 0xd7, 0x7d, 0xc0, + 0xb9, 0xe2, 0xc5, 0x3c, 0x2e, 0xb4, 0x01, 0x05, 0x1e, 0x19, 0xa0, 0xb4, 0xf5, 0x43, 0x2c, 0x2b, + 0xce, 0x79, 0x56, 0xb7, 0x1c, 0xe3, 0x5d, 0x21, 0xac, 0x2b, 0x0e, 0xd2, 0x20, 0xc3, 0x85, 0xe9, + 0x1a, 0x4f, 0xbf, 0xf3, 0xf4, 0xa7, 0xdb, 0x50, 0x4c, 0xb3, 0xf0, 0x84, 0xfa, 0x9a, 0x94, 0x66, + 0x92, 0xeb, 0x9a, 0xf8, 0x1f, 0x05, 0xb8, 0x35, 0x46, 0xc5, 0x71, 0x0e, 0xdd, 0x32, 0xa4, 0x0f, + 0xc9, 0x83, 0x74, 0xae, 0xe3, 0xb4, 0xe4, 0x5d, 0xa3, 0x4d, 0xc8, 0xed, 0x29, 0x7a, 0xdb, 0x1f, + 0xd2, 0xc3, 0xc3, 0x0b, 0xa3, 0x23, 0x5d, 0xe7, 0x18, 0x3b, 0x1b, 0xc3, 0xe2, 0x6f, 0x24, 0x60, + 0x7e, 0x45, 0xd3, 0x1a, 0x0d, 0x6e, 0x03, 0xe3, 0x1b, 0x29, 0x2e, 0xc8, 0x4c, 0xf8, 0x20, 0x13, + 0xbd, 0x07, 0x48, 0xd3, 0x6d, 0x96, 0xe6, 0xc3, 0xde, 0x57, 0x34, 0xf3, 0xc8, 0x0f, 0xab, 0x98, + 0x77, 0xef, 0x34, 0xdc, 0x1b, 0xa8, 0x01, 0x14, 0xed, 0xc8, 0xb6, 0xa3, 0x78, 0xef, 0x8d, 0x6e, + 0x4d, 0x74, 0x5c, 0x8b, 0xc1, 0x20, 0xef, 0x52, 0xca, 0x10, 0x39, 0xf4, 0x27, 0xf1, 0xdb, 0x75, + 0xd2, 0x74, 0x47, 0x56, 0x6c, 0xf7, 0x6c, 0x0e, 0x4b, 0x30, 0x92, 0x67, 0xf4, 0x15, 0x9b, 0x1d, + 0xb9, 0x61, 0x87, 0x09, 0x7c, 0xd5, 0xc4, 0x09, 0x89, 0xff, 0x8e, 0x00, 0x79, 0x09, 0xef, 0x59, + 0xd8, 0x8e, 0x75, 0x53, 0xe0, 0x09, 0xcc, 0x59, 0x4c, 0xaa, 0xbc, 0x67, 0x99, 0x9d, 0xf3, 0xcc, + 0xab, 0x2c, 0x67, 0x7c, 0x62, 0x99, 0x1d, 0x6e, 0x58, 0x5e, 0x42, 0xc1, 0xab, 0x63, 0x9c, 0x8d, + 0xff, 0x3d, 0x7a, 0x14, 0x99, 0x09, 0x8e, 0x3b, 0xbe, 0x21, 0x5e, 0x0d, 0xd0, 0x17, 0x55, 0xc1, + 0x8a, 0xc6, 0xa9, 0x86, 0xff, 0x2a, 0x40, 0xbe, 0xd1, 0xdb, 0x65, 0x79, 0xa4, 0xe2, 0xd3, 0x40, + 0x15, 0x32, 0x6d, 0xbc, 0xe7, 0xc8, 0x6f, 0x14, 0x06, 0x9f, 0x26, 0xac, 0xf4, 0x10, 0xc0, 0x53, + 0x00, 0x8b, 0x1e, 0x5f, 0xa3, 0x72, 0x92, 0xe7, 0x94, 0x93, 0xa1, 0xbc, 0x84, 0x2c, 0xfe, 0xc3, + 0x04, 0x14, 0xbc, 0x66, 0xc6, 0x69, 0x25, 0x5f, 0x85, 0xac, 0x43, 0xf2, 0x3c, 0xd6, 0x61, 0x9e, + 0x87, 0x74, 0x44, 0x5b, 0x88, 0x25, 0x58, 0xa0, 0x8e, 0x8b, 0xac, 0x74, 0xbb, 0x6d, 0xdd, 0x85, + 0xbb, 0xd4, 0xfe, 0xa4, 0xa4, 0x79, 0x7a, 0x6b, 0x85, 0xdd, 0xa1, 0x40, 0x97, 0x8c, 0xb9, 0x3d, + 0x0b, 0xe3, 0x2f, 0xb0, 0x4c, 0x91, 0xd7, 0x79, 0x42, 0x56, 0xb2, 0x8c, 0xb1, 0x41, 0xf8, 0xf8, + 0x98, 0x7b, 0x05, 0xf3, 0x54, 0xa7, 0x71, 0x1f, 0xbb, 0x15, 0xff, 0x56, 0x02, 0x50, 0x50, 0xf2, + 0x57, 0xd7, 0x17, 0x89, 0xf8, 0xfa, 0xe2, 0x5d, 0x40, 0x2c, 0x68, 0xd1, 0x96, 0xbb, 0xd8, 0x92, + 0x6d, 0xac, 0x9a, 0x3c, 0xc7, 0x91, 0x20, 0x15, 0xf9, 0x9d, 0x6d, 0x6c, 0x35, 0x28, 0x1d, 0x3d, + 0x06, 0xf0, 0x3d, 0x32, 0xbe, 0x60, 0x8c, 0x74, 0xc8, 0xa4, 0x8c, 0xe7, 0x8a, 0x89, 0xdf, 0x17, + 0x20, 0xbf, 0xa9, 0xb7, 0x2c, 0x25, 0xd6, 0xe4, 0x3d, 0xe8, 0xe3, 0xf0, 0x2e, 0x78, 0xf6, 0x61, + 0x39, 0x2a, 0xc0, 0x86, 0x95, 0x70, 0xb1, 0x15, 0x67, 0x20, 0x16, 0xd8, 0xab, 0x51, 0x9c, 0xa6, + 0xe7, 0xdf, 0x97, 0x61, 0x8e, 0xd7, 0x7b, 0xc7, 0xd0, 0x4d, 0x03, 0x3d, 0x80, 0x64, 0x8b, 0xbf, + 0xe5, 0xc8, 0x46, 0xee, 0x48, 0xfa, 0xf9, 0xea, 0x6a, 0x53, 0x12, 0x29, 0x4b, 0x58, 0xba, 0x3d, + 0x27, 0xc2, 0x07, 0xf4, 0x63, 0xce, 0x83, 0x2c, 0xdd, 0x9e, 0x83, 0x1a, 0x50, 0x50, 0xfd, 0x7c, + 0x5c, 0x32, 0x61, 0x4f, 0x0e, 0xc5, 0x8b, 0x91, 0x19, 0xd0, 0x6a, 0x53, 0x52, 0x5e, 0x0d, 0xdd, + 0x40, 0x95, 0x60, 0x1a, 0xa8, 0xd4, 0x40, 0x40, 0x9b, 0x7f, 0x94, 0x39, 0x9c, 0x82, 0xaa, 0x36, + 0x15, 0xc8, 0x16, 0x85, 0x3e, 0x86, 0x19, 0x8d, 0x26, 0x1c, 0xe2, 0x93, 0x37, 0xaa, 0xa3, 0x43, + 0x79, 0x9d, 0x6a, 0x53, 0x12, 0xe7, 0x40, 0xeb, 0x30, 0xc7, 0x7e, 0x31, 0x8f, 0x8c, 0x83, 0xe8, + 0x5b, 0xc3, 0x25, 0x04, 0xd6, 0xbc, 0xda, 0x94, 0x94, 0xd5, 0x7c, 0x2a, 0xfa, 0x26, 0xa4, 0x6c, + 0x55, 0x71, 0x61, 0xf4, 0xd5, 0x21, 0xd9, 0x43, 0x7c, 0x66, 0x5a, 0x1a, 0x3d, 0x66, 0x19, 0x28, + 0x9d, 0x63, 0x77, 0x47, 0x33, 0xaa, 0xfa, 0xa1, 0x53, 0xea, 0xa4, 0xfa, 0x98, 0x12, 0xd0, 0x53, + 0xc8, 0x2a, 0xc4, 0xb5, 0x95, 0xe9, 0x39, 0x4f, 0xba, 0x85, 0x19, 0x1d, 0x47, 0x30, 0x70, 0x2e, + 0xb7, 0x46, 0x8f, 0xc2, 0xbb, 0x44, 0x5f, 0x50, 0x07, 0x5b, 0x2d, 0x5c, 0xca, 0x8e, 0x16, 0x14, + 0x0c, 0x63, 0xf3, 0x04, 0x51, 0x22, 0x71, 0x71, 0xbd, 0x43, 0xd6, 0xb4, 0x51, 0x73, 0x43, 0xdf, + 0x5c, 0x47, 0x9c, 0x50, 0xaa, 0x4d, 0x49, 0x73, 0xfb, 0x01, 0x32, 0x5a, 0x82, 0x44, 0x4b, 0x2d, + 0xe5, 0x86, 0x1a, 0x03, 0xef, 0x14, 0x4e, 0x6d, 0x4a, 0x4a, 0xb4, 0x54, 0xf4, 0x29, 0xa4, 0xd9, + 0x91, 0x8a, 0x63, 0xa3, 0x94, 0x1f, 0x3a, 0xc7, 0xc2, 0x07, 0x53, 0x6a, 0x53, 0x12, 0x3d, 0xc5, + 0x41, 0x9e, 0xb7, 0x0d, 0x79, 0x8b, 0xc5, 0x01, 0xba, 0x51, 0xbb, 0xc5, 0xa1, 0x6f, 0xf3, 0xa3, + 0x02, 0x77, 0x6b, 0x14, 0xe4, 0x04, 0xe8, 0xe8, 0x3b, 0x70, 0x21, 0x2c, 0x91, 0x8f, 0xb4, 0xf9, + 0xa1, 0x6f, 0xa6, 0x87, 0x06, 0x91, 0xd6, 0xa6, 0x24, 0x64, 0x0d, 0xdc, 0x44, 0x1f, 0xc2, 0x34, + 0xeb, 0x35, 0x44, 0x45, 0x46, 0x05, 0xa8, 0xf4, 0x75, 0x18, 0x2b, 0x4f, 0x06, 0xbf, 0xc3, 0x83, + 0xe1, 0xe4, 0xb6, 0xd9, 0x2a, 0x2d, 0x0c, 0x1d, 0xfc, 0x83, 0x61, 0x7d, 0x64, 0xf0, 0x3b, 0x3e, + 0x95, 0xf4, 0xbb, 0xc5, 0xee, 0xf0, 0x08, 0xaa, 0x0b, 0x43, 0xfb, 0x3d, 0x22, 0x46, 0xae, 0x46, + 0x8f, 0x29, 0xf8, 0x64, 0x52, 0x35, 0x8b, 0x25, 0xbc, 0x91, 0xe9, 0x9c, 0xba, 0x38, 0xb4, 0x6a, + 0x83, 0x19, 0x81, 0x6a, 0xd4, 0x1d, 0xf4, 0xa8, 0xe8, 0x25, 0x14, 0x79, 0x5a, 0x0a, 0xff, 0xf5, + 0xc9, 0x25, 0x2a, 0xef, 0x5e, 0xa4, 0xe9, 0x8a, 0x0a, 0x3f, 0xaa, 0x4d, 0x49, 0x05, 0x35, 0x7c, + 0x07, 0x7d, 0x06, 0xf3, 0x54, 0x9e, 0xac, 0xfa, 0x99, 0x44, 0x4a, 0xa5, 0x81, 0xbc, 0x14, 0xc3, + 0x93, 0x8e, 0xb8, 0x92, 0x8b, 0x6a, 0xdf, 0x2d, 0x32, 0x8c, 0x75, 0x43, 0x77, 0xa8, 0x95, 0x2d, + 0x0f, 0x1d, 0xc6, 0xe1, 0xfc, 0x85, 0x64, 0x18, 0xeb, 0x8c, 0x42, 0x86, 0xb1, 0xc3, 0xc3, 0xeb, + 0x78, 0x77, 0xbc, 0x3d, 0x74, 0x18, 0x47, 0xc5, 0xe1, 0x91, 0x61, 0xec, 0x04, 0xe9, 0x64, 0x18, + 0x33, 0x03, 0xd1, 0x27, 0xf7, 0x9d, 0xa1, 0xc3, 0x78, 0xe8, 0x79, 0x6b, 0x32, 0x8c, 0x95, 0x81, + 0x9b, 0x68, 0x0d, 0x80, 0x79, 0x6e, 0x74, 0xfd, 0xbf, 0x3a, 0x74, 0x31, 0xe8, 0x0f, 0xb3, 0x23, + 0x8b, 0x41, 0xdb, 0xa5, 0x11, 0x43, 0x46, 0x71, 0xa1, 0x4c, 0xdf, 0x16, 0x97, 0x16, 0x87, 0x1a, + 0xb2, 0x81, 0x37, 0xbb, 0xc4, 0x90, 0x1d, 0x79, 0x44, 0xb2, 0xaa, 0xb0, 0xed, 0xed, 0xd2, 0xb5, + 0xe1, 0x66, 0x39, 0xf8, 0x96, 0x8b, 0x9a, 0x65, 0x4a, 0x40, 0x2b, 0x90, 0x21, 0xee, 0xcd, 0x09, + 0x35, 0x43, 0xd7, 0x87, 0xba, 0xdf, 0x7d, 0xe7, 0x70, 0x6a, 0x53, 0x52, 0xfa, 0x35, 0x27, 0x91, + 0xc7, 0xb3, 0x8d, 0xbf, 0x92, 0x38, 0xf4, 0xf1, 0xa1, 0x4d, 0x62, 0xf2, 0x78, 0xc6, 0x81, 0x54, + 0xb8, 0xc8, 0xfa, 0x8a, 0x1f, 0x7d, 0xb6, 0xf8, 0x69, 0xdd, 0xd2, 0x0d, 0x2a, 0x6a, 0xe8, 0x0e, + 0x5a, 0xe4, 0x89, 0xec, 0xda, 0x94, 0xb4, 0xa0, 0x0c, 0xde, 0x25, 0x13, 0x9e, 0x2f, 0x3d, 0x6c, + 0xdf, 0xad, 0x74, 0x73, 0xe8, 0x84, 0x8f, 0xd8, 0xae, 0x24, 0x13, 0x5e, 0x09, 0x90, 0xd9, 0x02, + 0xa4, 0xc9, 0xb6, 0xcd, 0x62, 0x0b, 0x6e, 0x8d, 0x58, 0x80, 0xfa, 0x36, 0x3c, 0xd8, 0x02, 0xa4, + 0x35, 0x18, 0x27, 0x11, 0xa4, 0xb6, 0xb1, 0x62, 0x71, 0x33, 0x7b, 0x7b, 0xa8, 0xa0, 0x81, 0x54, + 0x81, 0x44, 0x90, 0xea, 0x11, 0x89, 0xc3, 0x63, 0xb9, 0xf9, 0x6c, 0xb8, 0x6f, 0x7c, 0x67, 0xa8, + 0xc3, 0x13, 0x99, 0x70, 0x87, 0x38, 0x3c, 0x56, 0xe8, 0x06, 0xfa, 0x39, 0x98, 0xe5, 0x48, 0xb5, + 0x74, 0x77, 0x84, 0x0b, 0x18, 0xdc, 0x5c, 0x20, 0xf3, 0x9a, 0xf3, 0x30, 0x2b, 0xcb, 0x10, 0x32, + 0x6b, 0xde, 0xbd, 0x11, 0x56, 0x76, 0x00, 0xa4, 0x33, 0x2b, 0xeb, 0x93, 0x89, 0x95, 0x65, 0xe3, + 0x94, 0xaf, 0x75, 0xf7, 0x87, 0x5a, 0xd9, 0xc1, 0x53, 0x3e, 0xc4, 0xca, 0xbe, 0xf6, 0xa9, 0xa4, + 0x65, 0x36, 0x43, 0x8a, 0xa5, 0x6f, 0x0c, 0x6d, 0x59, 0x18, 0x32, 0x93, 0x96, 0x71, 0x1e, 0xd2, + 0x6d, 0xcc, 0xfb, 0x67, 0x9a, 0x7e, 0x77, 0x78, 0x7e, 0x81, 0x7e, 0x7c, 0x55, 0x73, 0xf7, 0x64, + 0x99, 0x86, 0x3d, 0x43, 0x65, 0xf1, 0xf3, 0xd5, 0x5c, 0x53, 0xef, 0x8d, 0x36, 0x54, 0x51, 0x87, + 0xc6, 0x3d, 0x43, 0x15, 0xba, 0x49, 0xab, 0xca, 0x8e, 0xd4, 0xd1, 0xf9, 0xbd, 0x34, 0x22, 0x15, + 0x42, 0xdf, 0xc1, 0x46, 0x5a, 0x55, 0x8f, 0xe8, 0x4f, 0xa1, 0x1e, 0xcb, 0xd3, 0x51, 0x5a, 0x1e, + 0x3d, 0x85, 0xc2, 0x99, 0x42, 0xbc, 0x29, 0xc4, 0xc9, 0xde, 0x9a, 0xe9, 0x7a, 0x18, 0xef, 0x8f, + 0x5e, 0x33, 0xfb, 0x5d, 0x0b, 0xb6, 0x66, 0x72, 0x9f, 0xe2, 0x2f, 0x0b, 0x70, 0x8d, 0xd5, 0x8d, + 0x6e, 0x5e, 0x9e, 0xc8, 0xde, 0x16, 0x70, 0xe0, 0x78, 0xc7, 0x03, 0xfa, 0x80, 0x0f, 0x87, 0x55, + 0x77, 0xcc, 0x96, 0x76, 0x6d, 0x4a, 0x7a, 0x47, 0x19, 0x55, 0x8e, 0x0c, 0xa9, 0x0e, 0x43, 0x50, + 0xa5, 0x87, 0x43, 0x87, 0x54, 0x18, 0xf5, 0x91, 0x21, 0xc5, 0x79, 0x56, 0x67, 0xf9, 0x0b, 0x64, + 0xef, 0x38, 0x6c, 0xa1, 0x58, 0x5c, 0x4f, 0xa5, 0x2f, 0x17, 0x4b, 0xeb, 0xa9, 0xf4, 0x95, 0x62, + 0x79, 0x3d, 0x95, 0x7e, 0xab, 0xf8, 0xb6, 0xf8, 0x8f, 0xca, 0x90, 0x73, 0x41, 0x17, 0x03, 0x54, + 0x0f, 0x83, 0x80, 0xea, 0xea, 0x30, 0x40, 0xc5, 0x61, 0x1a, 0x47, 0x54, 0x0f, 0x83, 0x88, 0xea, + 0xea, 0x30, 0x44, 0xe5, 0xf3, 0x10, 0x48, 0xd5, 0x1c, 0x06, 0xa9, 0xee, 0x4d, 0x00, 0xa9, 0x3c, + 0x51, 0xfd, 0x98, 0x6a, 0x6d, 0x10, 0x53, 0xdd, 0x1c, 0x8d, 0xa9, 0x3c, 0x51, 0x01, 0x50, 0xf5, + 0xb8, 0x0f, 0x54, 0x5d, 0x1f, 0x01, 0xaa, 0x3c, 0x7e, 0x17, 0x55, 0x6d, 0x44, 0xa2, 0xaa, 0xdb, + 0xe3, 0x50, 0x95, 0x27, 0x27, 0x04, 0xab, 0x3e, 0x08, 0xc1, 0xaa, 0xc5, 0xa1, 0xb0, 0xca, 0xe3, + 0x66, 0xb8, 0xea, 0x93, 0x7e, 0x5c, 0x75, 0x7d, 0x04, 0xae, 0xf2, 0x5b, 0xc0, 0x81, 0x55, 0x2d, + 0x0a, 0x58, 0xdd, 0x1a, 0x03, 0xac, 0x3c, 0x29, 0x41, 0x64, 0x55, 0x8b, 0x42, 0x56, 0xb7, 0xc6, + 0x20, 0xab, 0x3e, 0x49, 0x0c, 0x5a, 0x6d, 0x45, 0x43, 0xab, 0x3b, 0x63, 0xa1, 0x95, 0x27, 0x2d, + 0x8c, 0xad, 0x96, 0x03, 0xd8, 0xea, 0x9d, 0x21, 0xd8, 0xca, 0x63, 0x25, 0xe0, 0xea, 0x5b, 0x03, + 0xe0, 0x4a, 0x1c, 0x05, 0xae, 0x3c, 0x5e, 0x0f, 0x5d, 0xbd, 0x18, 0x82, 0xae, 0xee, 0x8e, 0x47, + 0x57, 0x9e, 0xb0, 0x3e, 0x78, 0xa5, 0x8c, 0x84, 0x57, 0xef, 0x4d, 0x08, 0xaf, 0x3c, 0xe9, 0x51, + 0xf8, 0xea, 0xa3, 0x30, 0xbe, 0xba, 0x36, 0x1c, 0x5f, 0x79, 0x62, 0x38, 0xc0, 0xda, 0x88, 0x04, + 0x58, 0xb7, 0xc7, 0x01, 0x2c, 0x7f, 0x1e, 0x04, 0x11, 0xd6, 0x56, 0x34, 0xc2, 0xba, 0x33, 0x16, + 0x61, 0xf9, 0xdd, 0x1f, 0x82, 0x58, 0x1b, 0x91, 0x10, 0xeb, 0xf6, 0x38, 0x88, 0xe5, 0x57, 0x2e, + 0x88, 0xb1, 0x5e, 0x0d, 0xc5, 0x58, 0xf7, 0x27, 0xc1, 0x58, 0x9e, 0xd0, 0x01, 0x90, 0xf5, 0xf9, + 0x70, 0x90, 0xf5, 0x8d, 0x73, 0x64, 0x76, 0x8c, 0x44, 0x59, 0xdf, 0x1a, 0x40, 0x59, 0xe2, 0x28, + 0x94, 0xe5, 0x8f, 0x67, 0x17, 0x66, 0x29, 0x23, 0x41, 0xd1, 0x7b, 0x13, 0x82, 0x22, 0x7f, 0xf0, + 0x45, 0xa0, 0xa2, 0x6a, 0x04, 0x2a, 0xba, 0x39, 0x1a, 0x15, 0xf9, 0xe6, 0xdc, 0x87, 0x45, 0xb5, + 0x28, 0x58, 0x74, 0x6b, 0x0c, 0x2c, 0xf2, 0xad, 0x50, 0x00, 0x17, 0x3d, 0xee, 0xc3, 0x45, 0xd7, + 0xc7, 0xc6, 0x47, 0x05, 0x80, 0xd1, 0xea, 0x20, 0x30, 0xba, 0x31, 0x12, 0x18, 0x79, 0x12, 0x7c, + 0x64, 0xf4, 0xb8, 0x0f, 0x19, 0x5d, 0x1f, 0x81, 0x8c, 0xfc, 0x0a, 0x70, 0x68, 0xa4, 0x8d, 0x86, + 0x46, 0x4b, 0x93, 0x42, 0x23, 0x4f, 0x70, 0x24, 0x36, 0xda, 0x8a, 0xc6, 0x46, 0x77, 0x26, 0x0c, + 0x5d, 0x18, 0x00, 0x47, 0xb5, 0x28, 0x70, 0x74, 0x6b, 0x0c, 0x38, 0x0a, 0xae, 0x21, 0x1e, 0x3a, + 0xaa, 0x45, 0xa1, 0xa3, 0x5b, 0x63, 0xd0, 0x91, 0x2f, 0x29, 0x00, 0x8f, 0x9a, 0xc3, 0xe0, 0xd1, + 0xbd, 0x09, 0xe0, 0x91, 0xef, 0xbc, 0xf4, 0xe1, 0xa3, 0x4f, 0xfb, 0xf1, 0x91, 0x38, 0x0a, 0x1f, + 0xf9, 0x33, 0xd2, 0x05, 0x48, 0x5b, 0xd1, 0x00, 0xe9, 0xce, 0x58, 0x80, 0x14, 0x34, 0x92, 0x01, + 0x84, 0xb4, 0x11, 0x89, 0x90, 0x6e, 0x8f, 0x43, 0x48, 0xbe, 0x91, 0x0c, 0x42, 0xa4, 0x4f, 0xfb, + 0x21, 0x92, 0x38, 0x0a, 0x22, 0xf9, 0x8d, 0x73, 0x31, 0x52, 0x2d, 0x0a, 0x23, 0xdd, 0x1a, 0x83, + 0x91, 0xfc, 0xce, 0x0b, 0x80, 0x24, 0x65, 0x24, 0x48, 0x7a, 0x6f, 0x42, 0x90, 0xd4, 0x67, 0xb8, + 0xc2, 0x28, 0xa9, 0x16, 0x85, 0x92, 0x6e, 0x8d, 0x41, 0x49, 0x81, 0xca, 0xfa, 0x30, 0x69, 0x2b, + 0x1a, 0x26, 0xdd, 0x19, 0x0b, 0x93, 0xfa, 0x66, 0x93, 0x8b, 0x93, 0x36, 0x22, 0x71, 0xd2, 0xed, + 0x71, 0x38, 0xa9, 0x6f, 0xe1, 0xe3, 0xce, 0xc1, 0xaf, 0x4c, 0x0e, 0x94, 0x3e, 0x3a, 0x3f, 0x50, + 0xf2, 0x9e, 0x39, 0x06, 0x29, 0x7d, 0xda, 0x8f, 0x94, 0xc4, 0x51, 0x48, 0xc9, 0x1f, 0x59, 0xe7, + 0x83, 0x4a, 0xeb, 0xa9, 0xf4, 0xdb, 0xc5, 0x77, 0xc4, 0xbf, 0x36, 0x0b, 0x33, 0x35, 0x2f, 0xa2, + 0xc8, 0x6f, 0xa5, 0xf0, 0x26, 0x19, 0xa8, 0xd0, 0x1a, 0x99, 0xf1, 0xd4, 0x6e, 0x8e, 0x4f, 0x2a, + 0x38, 0x98, 0xf6, 0x8e, 0xb3, 0xbe, 0xc1, 0x51, 0x70, 0xf4, 0x01, 0xe4, 0x7a, 0x36, 0xb6, 0xe4, + 0xae, 0xa5, 0x9b, 0x96, 0xee, 0xb0, 0xf3, 0x32, 0xc2, 0x6a, 0xf1, 0xcb, 0xd3, 0xc5, 0xb9, 0x1d, + 0x1b, 0x5b, 0xdb, 0x9c, 0x2e, 0xcd, 0xf5, 0x02, 0x57, 0xee, 0x27, 0xb9, 0xa6, 0x27, 0xff, 0x24, + 0xd7, 0x0b, 0x28, 0x5a, 0x58, 0xd1, 0x42, 0x1e, 0x0c, 0xcb, 0xf6, 0x14, 0x3d, 0xe6, 0xe8, 0x79, + 0x36, 0xb7, 0x24, 0xcd, 0xfa, 0x54, 0xb0, 0xc2, 0x44, 0xf4, 0x00, 0x2e, 0x76, 0x94, 0x63, 0x1a, + 0x7d, 0x2a, 0xbb, 0x4e, 0x21, 0x8d, 0x28, 0x65, 0x5f, 0xbb, 0x42, 0x1d, 0xe5, 0x98, 0x7e, 0xdf, + 0x8b, 0xdd, 0xa2, 0x1f, 0xed, 0xb8, 0x05, 0x79, 0x4d, 0xb7, 0x1d, 0xdd, 0x50, 0x1d, 0x9e, 0xe5, + 0x97, 0x65, 0xc8, 0xcd, 0xb9, 0x54, 0x96, 0xca, 0xf7, 0x3e, 0xcc, 0xf3, 0x63, 0x09, 0x81, 0x97, + 0xb1, 0x3c, 0x53, 0x2e, 0xbb, 0xe1, 0xbd, 0x7f, 0x45, 0x15, 0x28, 0xb4, 0x14, 0x07, 0x1f, 0x29, + 0x27, 0xb2, 0x7b, 0xe8, 0x2d, 0x4b, 0x93, 0x5f, 0xbe, 0x75, 0x76, 0xba, 0x98, 0x7b, 0xca, 0x6e, + 0x0d, 0x9c, 0x7d, 0xcb, 0xb5, 0x02, 0x37, 0x34, 0x74, 0x07, 0x0a, 0x8a, 0x7d, 0x62, 0xa8, 0x54, + 0x3d, 0xd8, 0xb0, 0x7b, 0x36, 0x85, 0x24, 0x69, 0x29, 0x4f, 0xc9, 0x15, 0x97, 0x8a, 0xae, 0xc3, + 0x1c, 0x8f, 0xd9, 0x67, 0x1f, 0x0c, 0x2a, 0xd0, 0xa6, 0xf2, 0x2f, 0x54, 0xd0, 0x6f, 0x06, 0xa1, + 0xc7, 0x50, 0xe6, 0x59, 0xfc, 0x8f, 0x14, 0x4b, 0x93, 0xa9, 0xd6, 0xfd, 0xf1, 0x59, 0xa4, 0x62, + 0x2f, 0xb3, 0xac, 0xfd, 0xa4, 0x00, 0x51, 0xb5, 0x3f, 0x87, 0xb6, 0x60, 0x5e, 0x6d, 0xeb, 0x1e, + 0x82, 0x60, 0x2d, 0x9f, 0x1f, 0x3a, 0x9b, 0x2a, 0xb4, 0xac, 0xff, 0x32, 0xba, 0xa0, 0x86, 0x09, + 0xa8, 0x01, 0x34, 0xaf, 0x8e, 0xdc, 0x35, 0xdb, 0xba, 0x7a, 0x42, 0xc1, 0x43, 0x38, 0x1b, 0xf9, + 0xc8, 0x6f, 0x02, 0xbc, 0x52, 0x74, 0x67, 0x9b, 0x72, 0x4a, 0x70, 0xe4, 0xfd, 0x66, 0x99, 0x84, + 0xd7, 0x53, 0xe9, 0xb9, 0x62, 0x6e, 0x3d, 0x95, 0xce, 0x17, 0x0b, 0xe2, 0x5f, 0x17, 0xa0, 0xd0, + 0x57, 0x17, 0x54, 0x83, 0x8b, 0x9a, 0x37, 0x55, 0x64, 0x7e, 0xa8, 0x4b, 0x37, 0x0d, 0x9e, 0x60, + 0x7d, 0xe1, 0xcb, 0xd3, 0xc5, 0x02, 0x2d, 0xfd, 0xd4, 0xbb, 0x25, 0x5d, 0xf0, 0x39, 0x7c, 0x2a, + 0xfa, 0x08, 0xf2, 0xcc, 0xfd, 0xf4, 0xbe, 0x81, 0x47, 0xa3, 0xf1, 0x57, 0xe7, 0xbf, 0x3c, 0x5d, + 0xcc, 0x51, 0x9f, 0xd3, 0x4d, 0x80, 0x2c, 0xe5, 0xda, 0xc1, 0x4b, 0xf1, 0xd7, 0x05, 0x98, 0x0b, + 0x1d, 0x9c, 0x7a, 0xdc, 0xf7, 0xf2, 0xfb, 0x4a, 0x34, 0x6e, 0x1d, 0x16, 0xba, 0x98, 0xe6, 0xe3, + 0xdc, 0x8d, 0x03, 0x5d, 0x1c, 0x8e, 0x7b, 0xe8, 0x2e, 0x8e, 0x1b, 0x14, 0xe3, 0xb2, 0x7d, 0x9c, + 0xfa, 0xc1, 0x0f, 0x17, 0xa7, 0xc4, 0xdf, 0x4f, 0x41, 0x2e, 0x7c, 0x4c, 0xaa, 0xde, 0x57, 0xaf, + 0xa8, 0x75, 0x25, 0xc4, 0xb1, 0x34, 0x22, 0x41, 0x64, 0xc6, 0xff, 0x92, 0x01, 0xab, 0xe6, 0xb5, + 0x11, 0xaf, 0xf8, 0x83, 0xf5, 0xf4, 0x19, 0xcb, 0xdf, 0x4b, 0x7a, 0xf6, 0x75, 0x09, 0xa6, 0x69, + 0xc2, 0x22, 0x5e, 0xb5, 0x52, 0xff, 0xe8, 0x21, 0xbe, 0x36, 0xb9, 0x2f, 0xb1, 0x62, 0xc4, 0x1e, + 0x37, 0xdf, 0x28, 0x23, 0xa0, 0x3f, 0x0d, 0xce, 0xff, 0xc9, 0x41, 0x9e, 0x11, 0x72, 0xfa, 0x7c, + 0x19, 0x21, 0xd1, 0x2f, 0x41, 0x41, 0x35, 0xdb, 0x6d, 0xb6, 0x56, 0x32, 0x8b, 0x34, 0x98, 0xe3, + 0x85, 0x8a, 0xe0, 0x5f, 0x89, 0x5c, 0xf2, 0xbe, 0x16, 0xb9, 0x24, 0xf1, 0xaf, 0x45, 0x06, 0xa2, + 0x6a, 0xf3, 0x9e, 0x30, 0x66, 0xc8, 0xfa, 0x02, 0x7c, 0x67, 0xdf, 0x24, 0xc0, 0x97, 0x05, 0x85, + 0xf3, 0x91, 0xf3, 0x27, 0x02, 0x0f, 0xc1, 0x79, 0x66, 0x9a, 0x07, 0x3d, 0x2f, 0x30, 0xb7, 0x1c, + 0xcc, 0xcf, 0x98, 0xfe, 0xf2, 0x74, 0x31, 0x25, 0x79, 0x09, 0x1a, 0xa3, 0x2c, 0x7f, 0xe2, 0xa7, + 0xb3, 0xfc, 0xd7, 0x61, 0xae, 0x6b, 0xe1, 0x3d, 0xec, 0xa8, 0xfb, 0xb2, 0xd1, 0xeb, 0xf0, 0xf3, + 0x3b, 0x59, 0x97, 0xb6, 0xd5, 0xeb, 0xa0, 0x7b, 0x50, 0xf4, 0x8a, 0x70, 0x8c, 0xee, 0x26, 0xf0, + 0x72, 0xe9, 0x1c, 0xd1, 0x8b, 0xff, 0x4b, 0x80, 0x85, 0x50, 0x9b, 0xf8, 0x9c, 0x58, 0x87, 0xac, + 0x6f, 0x0e, 0xec, 0x92, 0x70, 0xce, 0x30, 0xd5, 0x20, 0x33, 0x92, 0xe1, 0x92, 0xfb, 0x58, 0x9a, + 0xbd, 0xdf, 0x17, 0x9b, 0x38, 0xa7, 0xd8, 0x8b, 0xbe, 0x9c, 0xb5, 0xc0, 0x03, 0xbc, 0x49, 0x92, + 0x9c, 0x68, 0x92, 0x88, 0xbf, 0x25, 0x40, 0x91, 0x3e, 0xe0, 0x09, 0xc6, 0x5a, 0x2c, 0xd6, 0xc9, + 0x0d, 0xff, 0x4e, 0x4c, 0x7e, 0x42, 0x27, 0xf4, 0xc5, 0x91, 0x64, 0xf8, 0x8b, 0x23, 0xe2, 0x0f, + 0x05, 0xc8, 0x7b, 0x35, 0x64, 0xdf, 0xdd, 0x1b, 0x91, 0x06, 0xf4, 0xcd, 0xbe, 0x39, 0xe7, 0xe6, + 0x3b, 0x99, 0xe8, 0x53, 0x80, 0xc1, 0x7c, 0x27, 0xec, 0x5b, 0x69, 0x7f, 0xd3, 0x1d, 0x39, 0xa4, + 0x8a, 0x15, 0x3f, 0xaf, 0xc5, 0x1b, 0x1c, 0x56, 0x92, 0xe8, 0x27, 0x4b, 0xcd, 0xf6, 0x21, 0x4b, + 0x31, 0x33, 0x91, 0xd9, 0x42, 0x3c, 0xf0, 0x0c, 0xf8, 0x6e, 0x9e, 0xd6, 0x6c, 0xd0, 0x8f, 0x99, + 0xb2, 0xdf, 0xb6, 0xf8, 0x24, 0xa0, 0x40, 0xda, 0xf9, 0x44, 0x4b, 0x13, 0x99, 0x52, 0x57, 0x4b, + 0x6c, 0xac, 0xfc, 0x71, 0xb0, 0x27, 0xaa, 0x87, 0x04, 0xc5, 0x3d, 0x82, 0xe4, 0xa1, 0xd2, 0x1e, + 0x15, 0xc1, 0x15, 0xea, 0x39, 0x89, 0x94, 0x46, 0x4f, 0x42, 0xe9, 0x40, 0x12, 0xc3, 0x11, 0xc7, + 0xa0, 0x4a, 0x43, 0x69, 0x43, 0x3e, 0x0c, 0x8f, 0xf5, 0x91, 0x8f, 0x0f, 0x0e, 0xfa, 0x8f, 0x53, + 0x3f, 0xfa, 0xe1, 0xa2, 0x20, 0x7e, 0x02, 0x48, 0xc2, 0x36, 0x76, 0x5e, 0xf4, 0x4c, 0xcb, 0x4f, + 0xad, 0x72, 0xbb, 0xef, 0x13, 0x2c, 0xd3, 0xab, 0xd9, 0xb3, 0xa8, 0xe4, 0x49, 0x17, 0x61, 0x21, + 0xc4, 0xcd, 0x8c, 0x85, 0xf8, 0x21, 0x5c, 0x79, 0x6a, 0xda, 0xb6, 0xde, 0x25, 0xd0, 0x95, 0xce, + 0x4a, 0xb2, 0x34, 0x78, 0xe6, 0x31, 0xdd, 0xa5, 0x9b, 0x18, 0x06, 0x33, 0x23, 0x19, 0xc9, 0xbb, + 0x16, 0xff, 0x40, 0x80, 0xcb, 0x83, 0x9c, 0x4c, 0xcb, 0x51, 0x67, 0x2b, 0x67, 0x55, 0xd3, 0xcf, + 0xa4, 0x37, 0x7e, 0xb4, 0xba, 0xc5, 0x89, 0x8b, 0xc9, 0x9f, 0x29, 0x77, 0x14, 0x6a, 0x3e, 0xf8, + 0x39, 0xef, 0x3c, 0x27, 0x6f, 0x32, 0xaa, 0x6f, 0x49, 0x52, 0x93, 0x59, 0x92, 0x26, 0x14, 0xd6, + 0x4d, 0xdd, 0x20, 0x9e, 0xac, 0xdb, 0xde, 0x15, 0xc8, 0xef, 0xea, 0x86, 0x62, 0x9d, 0xc8, 0x6e, + 0xe0, 0xa0, 0x30, 0x2e, 0x70, 0x50, 0xca, 0x31, 0x0e, 0x7e, 0x29, 0xfe, 0x58, 0x80, 0xa2, 0x2f, + 0x96, 0x5b, 0xe4, 0x77, 0x01, 0xd4, 0x76, 0xcf, 0x76, 0xb0, 0xe5, 0xf6, 0xd2, 0x1c, 0x8b, 0x93, + 0xaf, 0x30, 0x6a, 0x7d, 0x4d, 0xca, 0xf0, 0x02, 0x75, 0x0d, 0xdd, 0x08, 0xa7, 0xa1, 0x98, 0x5e, + 0x85, 0xb3, 0x81, 0xe4, 0x13, 0xa4, 0xdb, 0x6d, 0xc7, 0xb4, 0x3c, 0xcc, 0xc4, 0xbb, 0xdd, 0x4d, + 0x5a, 0x4f, 0x4f, 0x4f, 0x93, 0x72, 0x2b, 0x90, 0x27, 0xcb, 0xfd, 0x21, 0xf6, 0x9a, 0x94, 0x1a, + 0xdf, 0x24, 0xc6, 0xe1, 0x36, 0xe9, 0x77, 0x89, 0x93, 0xca, 0x7a, 0xc3, 0xeb, 0xe1, 0x11, 0x16, + 0xed, 0x67, 0x82, 0xc9, 0x1f, 0x27, 0x4b, 0x8f, 0x49, 0x3d, 0x91, 0x6f, 0x41, 0xda, 0xfd, 0x18, + 0x34, 0x9f, 0x20, 0x57, 0x96, 0xd8, 0xd7, 0xa2, 0x97, 0xdc, 0xaf, 0x45, 0x2f, 0xad, 0xf1, 0x02, + 0xcc, 0x8c, 0xff, 0xe0, 0xbf, 0x2c, 0x0a, 0x92, 0xc7, 0x74, 0xbf, 0x41, 0x46, 0xf8, 0xc0, 0x2a, + 0x8c, 0xf2, 0x00, 0x81, 0xaf, 0xf5, 0xf0, 0xef, 0x03, 0xaf, 0xac, 0xc9, 0x3b, 0x5b, 0x95, 0xe7, + 0x9b, 0x9b, 0xf5, 0x66, 0xb3, 0xba, 0x56, 0x14, 0x50, 0x11, 0xe6, 0x42, 0xdf, 0xfa, 0x49, 0xb0, + 0x2f, 0x06, 0xdf, 0xff, 0x19, 0x00, 0xff, 0xb3, 0x61, 0x44, 0xd6, 0x46, 0xf5, 0x33, 0xf9, 0xe5, + 0xca, 0xb3, 0x9d, 0x6a, 0xa3, 0x38, 0x85, 0x10, 0xe4, 0x57, 0x57, 0x9a, 0x95, 0x9a, 0x2c, 0x55, + 0x1b, 0xdb, 0xcf, 0xb7, 0x1a, 0x55, 0xf7, 0x4b, 0xc3, 0xf7, 0xd7, 0x60, 0x2e, 0x98, 0x58, 0x08, + 0x2d, 0x40, 0xa1, 0x52, 0xab, 0x56, 0x36, 0xe4, 0x97, 0xf5, 0x15, 0xf9, 0xc5, 0x4e, 0x75, 0xa7, + 0x5a, 0x9c, 0xa2, 0x55, 0xa3, 0xc4, 0x27, 0x3b, 0xcf, 0x9e, 0x15, 0x05, 0x54, 0x80, 0x2c, 0xbb, + 0xa6, 0xdf, 0x05, 0x2a, 0x26, 0xee, 0x6f, 0x42, 0x36, 0x90, 0x40, 0x98, 0x3c, 0x6e, 0x7b, 0xa7, + 0x51, 0x93, 0x9b, 0xf5, 0xcd, 0x6a, 0xa3, 0xb9, 0xb2, 0xb9, 0xcd, 0x64, 0x50, 0xda, 0xca, 0xea, + 0x73, 0xa9, 0x59, 0x14, 0xbc, 0xeb, 0xe6, 0xf3, 0x9d, 0x4a, 0xcd, 0x6d, 0x86, 0x98, 0x4a, 0x27, + 0x8b, 0xc9, 0xfb, 0x7f, 0x49, 0x80, 0xcb, 0x43, 0x92, 0xec, 0xa0, 0x2c, 0xcc, 0xee, 0x18, 0x34, + 0x9b, 0x6a, 0x71, 0x0a, 0xe5, 0x02, 0x79, 0x76, 0x8a, 0x02, 0x4a, 0xb3, 0x4c, 0x27, 0xc5, 0x04, + 0x9a, 0x81, 0x44, 0xe3, 0x51, 0x31, 0x49, 0x6a, 0x1a, 0x48, 0x53, 0x53, 0x4c, 0xa1, 0x0c, 0x4f, + 0x90, 0x51, 0x9c, 0x46, 0x73, 0x7e, 0x9e, 0x8a, 0xe2, 0x0c, 0x11, 0xe5, 0xe5, 0x7b, 0x28, 0xce, + 0xde, 0xbf, 0x0e, 0x81, 0x33, 0xf5, 0x08, 0x60, 0xe6, 0x99, 0xe2, 0x60, 0xdb, 0x29, 0x4e, 0xa1, + 0x59, 0x48, 0xae, 0xb4, 0xdb, 0x45, 0xe1, 0xe1, 0xbf, 0x48, 0x41, 0xda, 0xfd, 0xfe, 0x0d, 0x7a, + 0x06, 0xd3, 0x6c, 0xeb, 0x7a, 0x71, 0xb8, 0x67, 0x4f, 0x27, 0x6f, 0xf9, 0xda, 0x38, 0xd7, 0x5f, + 0x9c, 0x42, 0xff, 0x1f, 0x64, 0x03, 0x1e, 0x13, 0x1a, 0xba, 0xfd, 0x16, 0xf2, 0x12, 0xcb, 0xb7, + 0xc7, 0x15, 0xf3, 0xe4, 0xbf, 0x82, 0x8c, 0x67, 0xc1, 0xd1, 0x8d, 0x51, 0xf6, 0xdd, 0x95, 0x3d, + 0x7a, 0x11, 0x20, 0x73, 0x4d, 0x9c, 0x7a, 0x5f, 0x40, 0x16, 0xa0, 0x41, 0x63, 0x8b, 0xa2, 0x02, + 0x22, 0x86, 0x5a, 0xf3, 0xf2, 0xfd, 0x89, 0x4a, 0xfb, 0xcf, 0x24, 0xca, 0xf2, 0x57, 0x8c, 0x68, + 0x65, 0x0d, 0xac, 0x47, 0xd1, 0xca, 0x8a, 0x58, 0x78, 0xa6, 0xd0, 0x0b, 0x48, 0x11, 0x4b, 0x89, + 0xa2, 0x7c, 0xc8, 0x3e, 0xcb, 0x5c, 0xbe, 0x31, 0xb2, 0x8c, 0x2b, 0x72, 0xf5, 0xde, 0x8f, 0xfe, + 0xfc, 0xea, 0xd4, 0x8f, 0xce, 0xae, 0x0a, 0x3f, 0x3e, 0xbb, 0x2a, 0xfc, 0xe9, 0xd9, 0x55, 0xe1, + 0xcf, 0xce, 0xae, 0x0a, 0xdf, 0xff, 0xc9, 0xd5, 0xa9, 0x1f, 0xff, 0xe4, 0xea, 0xd4, 0x9f, 0xfe, + 0xe4, 0xea, 0xd4, 0xe7, 0xb3, 0x9c, 0x7b, 0x77, 0x86, 0x1a, 0x95, 0x47, 0xff, 0x37, 0x00, 0x00, + 0xff, 0xff, 0x86, 0x26, 0x3f, 0x28, 0xbd, 0x7f, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index ab86df7a67b8..64ddd5d59672 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1666,6 +1666,26 @@ message RangeStatsResponse { RangeInfo range_info = 4; } +// MigrateRequest is used instruct all ranges overlapping with it to exercise +// any relevant (below-raft) migrations in order for its range state to conform +// to what's needed by the specified version. It's a core primitive used in our +// migrations infrastructure to phase out legacy code below raft. +// +// KV waits for this command to durably apply on all replicas before returning, +// guaranteeing to the caller that all pre-migration state has been completely +// purged from the system. +message MigrateRequest { + RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; + + // Version is used to select the specific migration to exercise. + Version version = 2 [(gogoproto.nullable) = false]; +} + +// MigrateResponse is the response to a Migrate operation. +message MigrateResponse { + ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; +} + // A RequestUnion contains exactly one of the requests. // The values added here must match those in ResponseUnion. // @@ -1717,6 +1737,7 @@ message RequestUnion { SubsumeRequest subsume = 43; RangeStatsRequest range_stats = 44; AdminVerifyProtectedTimestampRequest admin_verify_protected_timestamp = 49; + MigrateRequest migrate = 50; } reserved 8, 15, 23, 25, 27; } @@ -1768,6 +1789,7 @@ message ResponseUnion { SubsumeResponse subsume = 43; RangeStatsResponse range_stats = 44; AdminVerifyProtectedTimestampResponse admin_verify_protected_timestamp = 49; + MigrateResponse migrate = 50; } reserved 8, 15, 23, 25, 27, 28; } diff --git a/pkg/roachpb/batch_generated.go b/pkg/roachpb/batch_generated.go index 5f0b865c6e3c..99556c74f790 100644 --- a/pkg/roachpb/batch_generated.go +++ b/pkg/roachpb/batch_generated.go @@ -164,6 +164,8 @@ func (ru RequestUnion) GetInner() Request { return t.RangeStats case *RequestUnion_AdminVerifyProtectedTimestamp: return t.AdminVerifyProtectedTimestamp + case *RequestUnion_Migrate: + return t.Migrate default: return nil } @@ -258,6 +260,8 @@ func (ru ResponseUnion) GetInner() Response { return t.RangeStats case *ResponseUnion_AdminVerifyProtectedTimestamp: return t.AdminVerifyProtectedTimestamp + case *ResponseUnion_Migrate: + return t.Migrate default: return nil } @@ -423,6 +427,8 @@ func (ru *RequestUnion) MustSetInner(r Request) { union = &RequestUnion_RangeStats{t} case *AdminVerifyProtectedTimestampRequest: union = &RequestUnion_AdminVerifyProtectedTimestamp{t} + case *MigrateRequest: + union = &RequestUnion_Migrate{t} default: panic(fmt.Sprintf("unsupported type %T for %T", r, ru)) } @@ -520,13 +526,15 @@ func (ru *ResponseUnion) MustSetInner(r Response) { union = &ResponseUnion_RangeStats{t} case *AdminVerifyProtectedTimestampResponse: union = &ResponseUnion_AdminVerifyProtectedTimestamp{t} + case *MigrateResponse: + union = &ResponseUnion_Migrate{t} default: panic(fmt.Sprintf("unsupported type %T for %T", r, ru)) } ru.Value = union } -type reqCounts [44]int32 +type reqCounts [45]int32 // getReqCounts returns the number of times each // request type appears in the batch. @@ -622,6 +630,8 @@ func (ba *BatchRequest) getReqCounts() reqCounts { counts[42]++ case *RequestUnion_AdminVerifyProtectedTimestamp: counts[43]++ + case *RequestUnion_Migrate: + counts[44]++ default: panic(fmt.Sprintf("unsupported request: %+v", ru)) } @@ -674,6 +684,7 @@ var requestNames = []string{ "Subsume", "RngStats", "AdmVerifyProtectedTimestamp", + "Migrate", } // Summary prints a short summary of the requests in a batch. @@ -881,6 +892,10 @@ type adminVerifyProtectedTimestampResponseAlloc struct { union ResponseUnion_AdminVerifyProtectedTimestamp resp AdminVerifyProtectedTimestampResponse } +type migrateResponseAlloc struct { + union ResponseUnion_Migrate + resp MigrateResponse +} // CreateReply creates replies for each of the contained requests, wrapped in a // BatchResponse. The response objects are batch allocated to minimize @@ -935,6 +950,7 @@ func (ba *BatchRequest) CreateReply() *BatchResponse { var buf41 []subsumeResponseAlloc var buf42 []rangeStatsResponseAlloc var buf43 []adminVerifyProtectedTimestampResponseAlloc + var buf44 []migrateResponseAlloc for i, r := range ba.Requests { switch r.GetValue().(type) { @@ -1246,6 +1262,13 @@ func (ba *BatchRequest) CreateReply() *BatchResponse { buf43[0].union.AdminVerifyProtectedTimestamp = &buf43[0].resp br.Responses[i].Value = &buf43[0].union buf43 = buf43[1:] + case *RequestUnion_Migrate: + if buf44 == nil { + buf44 = make([]migrateResponseAlloc, counts[44]) + } + buf44[0].union.Migrate = &buf44[0].resp + br.Responses[i].Value = &buf44[0].union + buf44 = buf44[1:] default: panic(fmt.Sprintf("unsupported request: %+v", r)) } @@ -1344,6 +1367,8 @@ func CreateRequest(method Method) Request { return &RangeStatsRequest{} case AdminVerifyProtectedTimestamp: return &AdminVerifyProtectedTimestampRequest{} + case Migrate: + return &MigrateRequest{} default: panic(fmt.Sprintf("unsupported method: %+v", method)) } diff --git a/pkg/roachpb/method.go b/pkg/roachpb/method.go index baaeb32cf080..1c0ec736b7ec 100644 --- a/pkg/roachpb/method.go +++ b/pkg/roachpb/method.go @@ -141,6 +141,9 @@ const ( AdminScatter // AddSSTable links a file into the RocksDB log-structured merge-tree. AddSSTable + // Migrate updates the range state to conform to a specified cluster + // version. It is our main mechanism for phasing out legacy code below Raft. + Migrate // RecomputeStats applies a delta to a Range's MVCCStats to fix computational errors. RecomputeStats // Refresh verifies no writes to a key have occurred since the diff --git a/pkg/roachpb/method_string.go b/pkg/roachpb/method_string.go index 67c7b3157110..4a65fe983c61 100644 --- a/pkg/roachpb/method_string.go +++ b/pkg/roachpb/method_string.go @@ -46,18 +46,19 @@ func _() { _ = x[Import-35] _ = x[AdminScatter-36] _ = x[AddSSTable-37] - _ = x[RecomputeStats-38] - _ = x[Refresh-39] - _ = x[RefreshRange-40] - _ = x[Subsume-41] - _ = x[RangeStats-42] - _ = x[AdminVerifyProtectedTimestamp-43] - _ = x[NumMethods-44] + _ = x[Migrate-38] + _ = x[RecomputeStats-39] + _ = x[Refresh-40] + _ = x[RefreshRange-41] + _ = x[Subsume-42] + _ = x[RangeStats-43] + _ = x[AdminVerifyProtectedTimestamp-44] + _ = x[NumMethods-45] } -const _Method_name = "GetPutConditionalPutIncrementDeleteDeleteRangeClearRangeRevertRangeScanReverseScanEndTxnAdminSplitAdminUnsplitAdminMergeAdminTransferLeaseAdminChangeReplicasAdminRelocateRangeHeartbeatTxnGCPushTxnRecoverTxnQueryTxnQueryIntentResolveIntentResolveIntentRangeMergeTruncateLogRequestLeaseTransferLeaseLeaseInfoComputeChecksumCheckConsistencyInitPutWriteBatchExportImportAdminScatterAddSSTableRecomputeStatsRefreshRefreshRangeSubsumeRangeStatsAdminVerifyProtectedTimestampNumMethods" +const _Method_name = "GetPutConditionalPutIncrementDeleteDeleteRangeClearRangeRevertRangeScanReverseScanEndTxnAdminSplitAdminUnsplitAdminMergeAdminTransferLeaseAdminChangeReplicasAdminRelocateRangeHeartbeatTxnGCPushTxnRecoverTxnQueryTxnQueryIntentResolveIntentResolveIntentRangeMergeTruncateLogRequestLeaseTransferLeaseLeaseInfoComputeChecksumCheckConsistencyInitPutWriteBatchExportImportAdminScatterAddSSTableMigrateRecomputeStatsRefreshRefreshRangeSubsumeRangeStatsAdminVerifyProtectedTimestampNumMethods" -var _Method_index = [...]uint16{0, 3, 6, 20, 29, 35, 46, 56, 67, 71, 82, 88, 98, 110, 120, 138, 157, 175, 187, 189, 196, 206, 214, 225, 238, 256, 261, 272, 284, 297, 306, 321, 337, 344, 354, 360, 366, 378, 388, 402, 409, 421, 428, 438, 467, 477} +var _Method_index = [...]uint16{0, 3, 6, 20, 29, 35, 46, 56, 67, 71, 82, 88, 98, 110, 120, 138, 157, 175, 187, 189, 196, 206, 214, 225, 238, 256, 261, 272, 284, 297, 306, 321, 337, 344, 354, 360, 366, 378, 388, 395, 409, 416, 428, 435, 445, 474, 484} func (i Method) String() string { if i < 0 || i >= Method(len(_Method_index)-1) { diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 0ed16ca993dd..ab61ba27bd8e 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -305,6 +305,7 @@ var charts = []sectionDescription{ "distsender.rpc.initput.sent", "distsender.rpc.leaseinfo.sent", "distsender.rpc.merge.sent", + "distsender.rpc.migrate.sent", "distsender.rpc.pushtxn.sent", "distsender.rpc.put.sent", "distsender.rpc.queryintent.sent", From 00f1a419fe7a3b8103a35c15f66320af4ea9816c Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Tue, 8 Dec 2020 00:11:53 -0500 Subject: [PATCH 11/11] *: introduce `GCReplicas` and `FlushAllEngines` RPCs We introduce two new RPCs for the migration infrastructure to use. `GCReplicas` will be used to instruct the target node to process all GC-able replicas. `FlushAllEngines` will be used to instruct the target node to persist all in-memory state to disk. Both of these are necessary primitives for the migration infastructure. Specifically this comes up in the context of wanting the ensure that ranges where we've executed a ranged `Migrate` command over have no way of ever surfacing pre-migrated state. This can happen with older replicas in the replica GC queue and with applied state that is not yet persisted. We elaborate on both of these below: Motivation for `GCReplicas`: Currently we wait for the `Migrate` to have applied on all replicas of a range before returning to the caller. This does not include earlier incarnations of the range, possibly sitting idle in the replica GC queue. These replicas can still request leases, and go through the request evaluation paths, possibly tripping up assertions that check to see no pre-migrated state is found. For this reason we introduce the `GCReplicas` RPC that the migration manager can use to ensure all GC-able replicas are processed before declaring the specific cluster version bump complete. Motivation for `FlushAllEngines`: Like we mentioned above, KV currently waits for the `Migrate` command to have applied on all replicas before returning. With the applied state, there's no necessity to durably persist it (the representative version is already stored in the raft log). Out of an abundance of caution, and to really really ensure that no pre-migrated state is ever seen in the system, we provide the migration manager a mechanism to flush out all in-memory state to disk. This way the manager can guarantee that by the time a specific cluster version is bumped, all pre-migrated state from prior to that cluster version will have been fully purged from the system. --- The ideas here follow from our original prototype in #57445. Neither of these RPCs are currently wired up to anything. That'll happen in a future commit introducing the raft truncated state migration. Release note: None --- pkg/kv/kvserver/queue_helpers_testutil.go | 18 - pkg/kv/kvserver/store.go | 26 + pkg/kv/kvserver/testing_knobs.go | 3 + pkg/server/migration.go | 28 ++ pkg/server/migration_test.go | 34 ++ pkg/server/serverpb/migration.pb.go | 559 +++++++++++++++++++++- pkg/server/serverpb/migration.proto | 26 +- 7 files changed, 657 insertions(+), 37 deletions(-) diff --git a/pkg/kv/kvserver/queue_helpers_testutil.go b/pkg/kv/kvserver/queue_helpers_testutil.go index 31cf64f7154f..60bc8684cd35 100644 --- a/pkg/kv/kvserver/queue_helpers_testutil.go +++ b/pkg/kv/kvserver/queue_helpers_testutil.go @@ -14,7 +14,6 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/errors" ) // Code in this file is for testing usage only. It is exported only because it @@ -26,23 +25,6 @@ func (bq *baseQueue) testingAdd( return bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), priority) } -func forceScanAndProcess(s *Store, q *baseQueue) error { - // Check that the system config is available. It is needed by many queues. If - // it's not available, some queues silently fail to process any replicas, - // which is undesirable for this method. - if cfg := s.Gossip().GetSystemConfig(); cfg == nil { - return errors.Errorf("system config not available in gossip") - } - - newStoreReplicaVisitor(s).Visit(func(repl *Replica) bool { - q.maybeAdd(context.Background(), repl, s.cfg.Clock.Now()) - return true - }) - - q.DrainQueue(s.stopper) - return nil -} - func mustForceScanAndProcess(ctx context.Context, s *Store, q *baseQueue) { if err := forceScanAndProcess(s, q); err != nil { log.Fatalf(ctx, "%v", err) diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 4fe6d9a83221..a501b272cbda 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -2763,6 +2763,32 @@ func (s *Store) ManuallyEnqueue( return collect(), processErr, nil } +// GCReplicas iterates over all ranges and processes any that may need to be +// GC'd. +func (s *Store) GCReplicas() error { + if interceptor := s.TestingKnobs().GCReplicasInterceptor; interceptor != nil { + interceptor() + } + return forceScanAndProcess(s, s.replicaGCQueue.baseQueue) +} + +func forceScanAndProcess(s *Store, q *baseQueue) error { + // Check that the system config is available. It is needed by many queues. If + // it's not available, some queues silently fail to process any replicas, + // which is undesirable for this method. + if cfg := s.Gossip().GetSystemConfig(); cfg == nil { + return errors.Errorf("system config not available in gossip") + } + + newStoreReplicaVisitor(s).Visit(func(repl *Replica) bool { + q.maybeAdd(context.Background(), repl, s.cfg.Clock.Now()) + return true + }) + + q.DrainQueue(s.stopper) + return nil +} + // WriteClusterVersion writes the given cluster version to the store-local // cluster version key. func WriteClusterVersion( diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index 2fdc13fb1706..5e45e275d594 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -259,6 +259,9 @@ type StoreTestingKnobs struct { // in execChangeReplicasTxn that prevent moving // to a configuration that cannot make progress. AllowDangerousReplicationChanges bool + // GCReplicasInterceptor intercepts attempts to GC all replicas in the + // store. + GCReplicasInterceptor func() } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. diff --git a/pkg/server/migration.go b/pkg/server/migration.go index 76765bac3db6..a2502fd2ea1f 100644 --- a/pkg/server/migration.go +++ b/pkg/server/migration.go @@ -143,3 +143,31 @@ func (m *migrationServer) BumpClusterVersion( resp := &serverpb.BumpClusterVersionResponse{} return resp, nil } + +// FlushAllEngines implements the MigrationServer interface. +func (m *migrationServer) FlushAllEngines( + _ context.Context, _ *serverpb.FlushAllEnginesRequest, +) (*serverpb.FlushAllEnginesResponse, error) { + for _, eng := range m.server.engines { + if err := eng.Flush(); err != nil { + return nil, err + } + } + + resp := &serverpb.FlushAllEnginesResponse{} + return resp, nil +} + +// GCReplicas implements the MigrationServer interface. +func (m *migrationServer) GCReplicas( + _ context.Context, _ *serverpb.GCReplicasRequest, +) (*serverpb.GCReplicasResponse, error) { + if err := m.server.node.stores.VisitStores(func(s *kvserver.Store) error { + return s.GCReplicas() + }); err != nil { + return nil, err + } + + resp := &serverpb.GCReplicasResponse{} + return resp, nil +} diff --git a/pkg/server/migration_test.go b/pkg/server/migration_test.go index 0056a4d8985e..6c8e74b03bec 100644 --- a/pkg/server/migration_test.go +++ b/pkg/server/migration_test.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -96,3 +97,36 @@ func TestValidateTargetClusterVersion(t *testing.T) { s.Stopper().Stop(context.Background()) } } + +func TestMigrationGCReplicas(t *testing.T) { + defer leaktest.AfterTest(t)() + + const numStores = 3 + var storeSpecs []base.StoreSpec + for i := 0; i < numStores; i++ { + storeSpecs = append(storeSpecs, base.StoreSpec{InMemory: true}) + } + + intercepted := 0 + s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ + StoreSpecs: storeSpecs, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + GCReplicasInterceptor: func() { + intercepted++ + }, + }, + }, + }) + + migrationServer := s.MigrationServer().(*migrationServer) + if _, err := migrationServer.GCReplicas(context.Background(), &serverpb.GCReplicasRequest{}); err != nil { + t.Fatal(err) + } + + if intercepted != numStores { + t.Fatalf("expected to have GC-ed replicas on %d stores, found %d", numStores, intercepted) + } + + s.Stopper().Stop(context.Background()) +} diff --git a/pkg/server/serverpb/migration.pb.go b/pkg/server/serverpb/migration.pb.go index e2eba51c12f3..3dc79f4664b6 100644 --- a/pkg/server/serverpb/migration.pb.go +++ b/pkg/server/serverpb/migration.pb.go @@ -36,7 +36,7 @@ func (m *ValidateTargetClusterVersionRequest) Reset() { *m = ValidateTar func (m *ValidateTargetClusterVersionRequest) String() string { return proto.CompactTextString(m) } func (*ValidateTargetClusterVersionRequest) ProtoMessage() {} func (*ValidateTargetClusterVersionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_migration_8dfeb6fcf9144e4c, []int{0} + return fileDescriptor_migration_02bb88ff52938b97, []int{0} } func (m *ValidateTargetClusterVersionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -70,7 +70,7 @@ func (m *ValidateTargetClusterVersionResponse) Reset() { *m = ValidateTa func (m *ValidateTargetClusterVersionResponse) String() string { return proto.CompactTextString(m) } func (*ValidateTargetClusterVersionResponse) ProtoMessage() {} func (*ValidateTargetClusterVersionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_migration_8dfeb6fcf9144e4c, []int{1} + return fileDescriptor_migration_02bb88ff52938b97, []int{1} } func (m *ValidateTargetClusterVersionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -95,8 +95,8 @@ func (m *ValidateTargetClusterVersionResponse) XXX_DiscardUnknown() { var xxx_messageInfo_ValidateTargetClusterVersionResponse proto.InternalMessageInfo -// BumpClusterVersionRequest is used to inform a given node of a cluster version -// bump. +// BumpClusterVersionRequest is used to inform the target node of a cluster +// version bump. type BumpClusterVersionRequest struct { ClusterVersion *clusterversion.ClusterVersion `protobuf:"bytes,1,opt,name=cluster_version,json=clusterVersion,proto3" json:"cluster_version,omitempty"` } @@ -105,7 +105,7 @@ func (m *BumpClusterVersionRequest) Reset() { *m = BumpClusterVersionReq func (m *BumpClusterVersionRequest) String() string { return proto.CompactTextString(m) } func (*BumpClusterVersionRequest) ProtoMessage() {} func (*BumpClusterVersionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_migration_8dfeb6fcf9144e4c, []int{2} + return fileDescriptor_migration_02bb88ff52938b97, []int{2} } func (m *BumpClusterVersionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -138,7 +138,7 @@ func (m *BumpClusterVersionResponse) Reset() { *m = BumpClusterVersionRe func (m *BumpClusterVersionResponse) String() string { return proto.CompactTextString(m) } func (*BumpClusterVersionResponse) ProtoMessage() {} func (*BumpClusterVersionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_migration_8dfeb6fcf9144e4c, []int{3} + return fileDescriptor_migration_02bb88ff52938b97, []int{3} } func (m *BumpClusterVersionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -163,11 +163,149 @@ func (m *BumpClusterVersionResponse) XXX_DiscardUnknown() { var xxx_messageInfo_BumpClusterVersionResponse proto.InternalMessageInfo +// GCReplicasRequest is used to instruct the target node to process all GC-able +// replicas. +type GCReplicasRequest struct { +} + +func (m *GCReplicasRequest) Reset() { *m = GCReplicasRequest{} } +func (m *GCReplicasRequest) String() string { return proto.CompactTextString(m) } +func (*GCReplicasRequest) ProtoMessage() {} +func (*GCReplicasRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_migration_02bb88ff52938b97, []int{4} +} +func (m *GCReplicasRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GCReplicasRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GCReplicasRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GCReplicasRequest.Merge(dst, src) +} +func (m *GCReplicasRequest) XXX_Size() int { + return m.Size() +} +func (m *GCReplicasRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GCReplicasRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_GCReplicasRequest proto.InternalMessageInfo + +// GCReplicasResponse is the response to a GCReplicasRequest. +type GCReplicasResponse struct { +} + +func (m *GCReplicasResponse) Reset() { *m = GCReplicasResponse{} } +func (m *GCReplicasResponse) String() string { return proto.CompactTextString(m) } +func (*GCReplicasResponse) ProtoMessage() {} +func (*GCReplicasResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_migration_02bb88ff52938b97, []int{5} +} +func (m *GCReplicasResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GCReplicasResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GCReplicasResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_GCReplicasResponse.Merge(dst, src) +} +func (m *GCReplicasResponse) XXX_Size() int { + return m.Size() +} +func (m *GCReplicasResponse) XXX_DiscardUnknown() { + xxx_messageInfo_GCReplicasResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_GCReplicasResponse proto.InternalMessageInfo + +// FlushAllEnginesRequest is used to instruct the target node to flush all its +// engines. +type FlushAllEnginesRequest struct { +} + +func (m *FlushAllEnginesRequest) Reset() { *m = FlushAllEnginesRequest{} } +func (m *FlushAllEnginesRequest) String() string { return proto.CompactTextString(m) } +func (*FlushAllEnginesRequest) ProtoMessage() {} +func (*FlushAllEnginesRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_migration_02bb88ff52938b97, []int{6} +} +func (m *FlushAllEnginesRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *FlushAllEnginesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *FlushAllEnginesRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_FlushAllEnginesRequest.Merge(dst, src) +} +func (m *FlushAllEnginesRequest) XXX_Size() int { + return m.Size() +} +func (m *FlushAllEnginesRequest) XXX_DiscardUnknown() { + xxx_messageInfo_FlushAllEnginesRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_FlushAllEnginesRequest proto.InternalMessageInfo + +// FlushAllEnginesResponse is the response to a FlushAllEnginesRequest. +type FlushAllEnginesResponse struct { +} + +func (m *FlushAllEnginesResponse) Reset() { *m = FlushAllEnginesResponse{} } +func (m *FlushAllEnginesResponse) String() string { return proto.CompactTextString(m) } +func (*FlushAllEnginesResponse) ProtoMessage() {} +func (*FlushAllEnginesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_migration_02bb88ff52938b97, []int{7} +} +func (m *FlushAllEnginesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *FlushAllEnginesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *FlushAllEnginesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_FlushAllEnginesResponse.Merge(dst, src) +} +func (m *FlushAllEnginesResponse) XXX_Size() int { + return m.Size() +} +func (m *FlushAllEnginesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_FlushAllEnginesResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_FlushAllEnginesResponse proto.InternalMessageInfo + func init() { proto.RegisterType((*ValidateTargetClusterVersionRequest)(nil), "cockroach.server.serverpb.ValidateTargetClusterVersionRequest") proto.RegisterType((*ValidateTargetClusterVersionResponse)(nil), "cockroach.server.serverpb.ValidateTargetClusterVersionResponse") proto.RegisterType((*BumpClusterVersionRequest)(nil), "cockroach.server.serverpb.BumpClusterVersionRequest") proto.RegisterType((*BumpClusterVersionResponse)(nil), "cockroach.server.serverpb.BumpClusterVersionResponse") + proto.RegisterType((*GCReplicasRequest)(nil), "cockroach.server.serverpb.GCReplicasRequest") + proto.RegisterType((*GCReplicasResponse)(nil), "cockroach.server.serverpb.GCReplicasResponse") + proto.RegisterType((*FlushAllEnginesRequest)(nil), "cockroach.server.serverpb.FlushAllEnginesRequest") + proto.RegisterType((*FlushAllEnginesResponse)(nil), "cockroach.server.serverpb.FlushAllEnginesResponse") } // Reference imports to suppress errors if they are not otherwise used. @@ -196,6 +334,12 @@ type MigrationClient interface { // which checks to see that all nodes in the cluster are running binaries // that would be able to support the intended version bump. BumpClusterVersion(ctx context.Context, in *BumpClusterVersionRequest, opts ...grpc.CallOption) (*BumpClusterVersionResponse, error) + // FlushAllEngines is used to instruct the target node to flush all its + // engines. + FlushAllEngines(ctx context.Context, in *FlushAllEnginesRequest, opts ...grpc.CallOption) (*FlushAllEnginesResponse, error) + // GCReplicas is used to instruct the target node to process all GC-able + // replicas. + GCReplicas(ctx context.Context, in *GCReplicasRequest, opts ...grpc.CallOption) (*GCReplicasResponse, error) } type migrationClient struct { @@ -224,6 +368,24 @@ func (c *migrationClient) BumpClusterVersion(ctx context.Context, in *BumpCluste return out, nil } +func (c *migrationClient) FlushAllEngines(ctx context.Context, in *FlushAllEnginesRequest, opts ...grpc.CallOption) (*FlushAllEnginesResponse, error) { + out := new(FlushAllEnginesResponse) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Migration/FlushAllEngines", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *migrationClient) GCReplicas(ctx context.Context, in *GCReplicasRequest, opts ...grpc.CallOption) (*GCReplicasResponse, error) { + out := new(GCReplicasResponse) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Migration/GCReplicas", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // MigrationServer is the server API for Migration service. type MigrationServer interface { // ValidateTargetClusterVersion is used to verify that the target node is @@ -240,6 +402,12 @@ type MigrationServer interface { // which checks to see that all nodes in the cluster are running binaries // that would be able to support the intended version bump. BumpClusterVersion(context.Context, *BumpClusterVersionRequest) (*BumpClusterVersionResponse, error) + // FlushAllEngines is used to instruct the target node to flush all its + // engines. + FlushAllEngines(context.Context, *FlushAllEnginesRequest) (*FlushAllEnginesResponse, error) + // GCReplicas is used to instruct the target node to process all GC-able + // replicas. + GCReplicas(context.Context, *GCReplicasRequest) (*GCReplicasResponse, error) } func RegisterMigrationServer(s *grpc.Server, srv MigrationServer) { @@ -282,6 +450,42 @@ func _Migration_BumpClusterVersion_Handler(srv interface{}, ctx context.Context, return interceptor(ctx, in, info, handler) } +func _Migration_FlushAllEngines_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(FlushAllEnginesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(MigrationServer).FlushAllEngines(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/cockroach.server.serverpb.Migration/FlushAllEngines", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(MigrationServer).FlushAllEngines(ctx, req.(*FlushAllEnginesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Migration_GCReplicas_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GCReplicasRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(MigrationServer).GCReplicas(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/cockroach.server.serverpb.Migration/GCReplicas", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(MigrationServer).GCReplicas(ctx, req.(*GCReplicasRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _Migration_serviceDesc = grpc.ServiceDesc{ ServiceName: "cockroach.server.serverpb.Migration", HandlerType: (*MigrationServer)(nil), @@ -294,6 +498,14 @@ var _Migration_serviceDesc = grpc.ServiceDesc{ MethodName: "BumpClusterVersion", Handler: _Migration_BumpClusterVersion_Handler, }, + { + MethodName: "FlushAllEngines", + Handler: _Migration_FlushAllEngines_Handler, + }, + { + MethodName: "GCReplicas", + Handler: _Migration_GCReplicas_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "server/serverpb/migration.proto", @@ -391,6 +603,78 @@ func (m *BumpClusterVersionResponse) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *GCReplicasRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GCReplicasRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + +func (m *GCReplicasResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GCReplicasResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + +func (m *FlushAllEnginesRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *FlushAllEnginesRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + +func (m *FlushAllEnginesResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *FlushAllEnginesResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + func encodeVarintMigration(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -444,6 +728,42 @@ func (m *BumpClusterVersionResponse) Size() (n int) { return n } +func (m *GCReplicasRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + +func (m *GCReplicasResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + +func (m *FlushAllEnginesRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + +func (m *FlushAllEnginesResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + func sovMigration(x uint64) (n int) { for { n++ @@ -723,6 +1043,206 @@ func (m *BumpClusterVersionResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *GCReplicasRequest) Unmarshal(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 ErrIntOverflowMigration + } + 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: GCReplicasRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GCReplicasRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipMigration(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMigration + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GCReplicasResponse) Unmarshal(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 ErrIntOverflowMigration + } + 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: GCReplicasResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GCReplicasResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipMigration(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMigration + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *FlushAllEnginesRequest) Unmarshal(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 ErrIntOverflowMigration + } + 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: FlushAllEnginesRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: FlushAllEnginesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipMigration(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMigration + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *FlushAllEnginesResponse) Unmarshal(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 ErrIntOverflowMigration + } + 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: FlushAllEnginesResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: FlushAllEnginesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipMigration(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMigration + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipMigration(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 @@ -829,11 +1349,11 @@ var ( ) func init() { - proto.RegisterFile("server/serverpb/migration.proto", fileDescriptor_migration_8dfeb6fcf9144e4c) + proto.RegisterFile("server/serverpb/migration.proto", fileDescriptor_migration_02bb88ff52938b97) } -var fileDescriptor_migration_8dfeb6fcf9144e4c = []byte{ - // 278 bytes of a gzipped FileDescriptorProto +var fileDescriptor_migration_02bb88ff52938b97 = []byte{ + // 366 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2f, 0x4e, 0x2d, 0x2a, 0x4b, 0x2d, 0xd2, 0x87, 0x50, 0x05, 0x49, 0xfa, 0xb9, 0x99, 0xe9, 0x45, 0x89, 0x25, 0x99, 0xf9, 0x79, 0x7a, 0x05, 0x45, 0xf9, 0x25, 0xf9, 0x42, 0x92, 0xc9, 0xf9, 0xc9, 0xd9, 0x45, 0xf9, 0x89, @@ -844,12 +1364,17 @@ var fileDescriptor_migration_8dfeb6fcf9144e4c = []byte{ 0xd3, 0x2f, 0xc1, 0xa8, 0xc0, 0xa8, 0xc1, 0x6d, 0xa4, 0xa1, 0x87, 0x70, 0x01, 0xaa, 0x85, 0x7a, 0x68, 0x26, 0xf1, 0x25, 0xa3, 0xf0, 0x95, 0xd4, 0xb8, 0x54, 0xf0, 0xdb, 0x5c, 0x5c, 0x90, 0x9f, 0x57, 0x9c, 0xaa, 0x94, 0xc7, 0x25, 0xe9, 0x54, 0x9a, 0x5b, 0x40, 0x37, 0x77, 0xc9, 0x70, 0x49, - 0x61, 0xb3, 0x0f, 0xe2, 0x1a, 0xa3, 0xad, 0x4c, 0x5c, 0x9c, 0xbe, 0xb0, 0x48, 0x10, 0x5a, 0xc8, - 0xc8, 0x25, 0x83, 0xcf, 0x13, 0x42, 0x76, 0x7a, 0x38, 0x23, 0x48, 0x8f, 0x88, 0x70, 0x97, 0xb2, - 0x27, 0x5b, 0x3f, 0x34, 0xf4, 0x18, 0x84, 0x9a, 0x19, 0xb9, 0x84, 0x30, 0x3d, 0x24, 0x64, 0x82, - 0xc7, 0x64, 0x9c, 0xe1, 0x2d, 0x65, 0x4a, 0xa2, 0x2e, 0x98, 0x2b, 0x9c, 0xb4, 0x4e, 0x3c, 0x94, - 0x63, 0x38, 0xf1, 0x48, 0x8e, 0xf1, 0xc2, 0x23, 0x39, 0xc6, 0x1b, 0x8f, 0xe4, 0x18, 0x1f, 0x3c, - 0x92, 0x63, 0x9c, 0xf0, 0x58, 0x8e, 0xe1, 0xc2, 0x63, 0x39, 0x86, 0x1b, 0x8f, 0xe5, 0x18, 0xa2, - 0x38, 0x60, 0x06, 0x25, 0xb1, 0x81, 0x93, 0xa6, 0x31, 0x20, 0x00, 0x00, 0xff, 0xff, 0x08, 0xa5, - 0xf0, 0xc3, 0xfe, 0x02, 0x00, 0x00, + 0x61, 0xb3, 0x0f, 0xea, 0x1a, 0x61, 0x2e, 0x41, 0x77, 0xe7, 0xa0, 0xd4, 0x82, 0x9c, 0xcc, 0xe4, + 0xc4, 0x62, 0xa8, 0x2b, 0x94, 0x44, 0xb8, 0x84, 0x90, 0x05, 0xa1, 0x4a, 0x25, 0xb8, 0xc4, 0xdc, + 0x72, 0x4a, 0x8b, 0x33, 0x1c, 0x73, 0x72, 0x5c, 0xf3, 0xd2, 0x33, 0xf3, 0x52, 0xe1, 0xea, 0x25, + 0xb9, 0xc4, 0x31, 0x64, 0x20, 0x9a, 0x8c, 0xe6, 0xb1, 0x70, 0x71, 0xfa, 0xc2, 0x22, 0x59, 0x68, + 0x21, 0x23, 0x97, 0x0c, 0xbe, 0x40, 0x12, 0xb2, 0xd3, 0xc3, 0x99, 0x00, 0xf4, 0x88, 0x88, 0x57, + 0x29, 0x7b, 0xb2, 0xf5, 0x43, 0x3d, 0xc9, 0x20, 0xd4, 0xcc, 0xc8, 0x25, 0x84, 0x19, 0x60, 0x42, + 0x26, 0x78, 0x4c, 0xc6, 0x19, 0x9f, 0x52, 0xa6, 0x24, 0xea, 0x82, 0xbb, 0xa2, 0x8a, 0x8b, 0x1f, + 0x2d, 0x48, 0x85, 0x0c, 0xf1, 0x98, 0x85, 0x3d, 0x62, 0xa4, 0x8c, 0x48, 0xd1, 0x02, 0xb7, 0x3b, + 0x9b, 0x8b, 0x0b, 0x11, 0xfd, 0x42, 0x3a, 0x78, 0xcc, 0xc0, 0x48, 0x3a, 0x52, 0xba, 0x44, 0xaa, + 0x86, 0x59, 0xe6, 0xa4, 0x75, 0xe2, 0xa1, 0x1c, 0xc3, 0x89, 0x47, 0x72, 0x8c, 0x17, 0x1e, 0xc9, + 0x31, 0xde, 0x78, 0x24, 0xc7, 0xf8, 0xe0, 0x91, 0x1c, 0xe3, 0x84, 0xc7, 0x72, 0x0c, 0x17, 0x1e, + 0xcb, 0x31, 0xdc, 0x78, 0x2c, 0xc7, 0x10, 0xc5, 0x01, 0x33, 0x20, 0x89, 0x0d, 0x9c, 0xc7, 0x8d, + 0x01, 0x01, 0x00, 0x00, 0xff, 0xff, 0xc5, 0x1c, 0x4a, 0x9f, 0x47, 0x04, 0x00, 0x00, } diff --git a/pkg/server/serverpb/migration.proto b/pkg/server/serverpb/migration.proto index 871b00dc8543..fd8d3394e11b 100644 --- a/pkg/server/serverpb/migration.proto +++ b/pkg/server/serverpb/migration.proto @@ -25,8 +25,8 @@ message ValidateTargetClusterVersionRequest { message ValidateTargetClusterVersionResponse { } -// BumpClusterVersionRequest is used to inform a given node of a cluster version -// bump. +// BumpClusterVersionRequest is used to inform the target node of a cluster +// version bump. message BumpClusterVersionRequest { clusterversion.ClusterVersion cluster_version = 1; } @@ -34,6 +34,20 @@ message BumpClusterVersionRequest { // BumpClusterVersionResponse is the response to an BumpClusterVersionRequest. message BumpClusterVersionResponse { } +// GCReplicasRequest is used to instruct the target node to process all GC-able +// replicas. +message GCReplicasRequest{} + +// GCReplicasResponse is the response to a GCReplicasRequest. +message GCReplicasResponse{} + +// FlushAllEnginesRequest is used to instruct the target node to flush all its +// engines. +message FlushAllEnginesRequest{} + +// FlushAllEnginesResponse is the response to a FlushAllEnginesRequest. +message FlushAllEnginesResponse{} + service Migration { // ValidateTargetClusterVersion is used to verify that the target node is // running a binary that's able to support the specified cluster version. @@ -50,4 +64,12 @@ service Migration { // which checks to see that all nodes in the cluster are running binaries // that would be able to support the intended version bump. rpc BumpClusterVersion(BumpClusterVersionRequest) returns (BumpClusterVersionResponse) { } + + // FlushAllEngines is used to instruct the target node to flush all its + // engines. + rpc FlushAllEngines (FlushAllEnginesRequest) returns (FlushAllEnginesResponse) { } + + // GCReplicas is used to instruct the target node to process all GC-able + // replicas. + rpc GCReplicas (GCReplicasRequest) returns (GCReplicasResponse) { } }