From 89781b1cd41e7913a08ba459fe63b237a90a4d6e Mon Sep 17 00:00:00 2001 From: irfan sharif Date: Mon, 7 Dec 2020 15:09:08 -0500 Subject: [PATCH] migration: introduce primitives for below-raft migrations... and onboard the first long running migration (for the truncated state, see below). The primitives here include: - 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. - The `SyncAllEngines` RPC. This is be used to instruct the target node to persist releveant in-memory state to disk. 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 will let us guarantee that by the time a specific cluster version is bumped, all pre-migrated state from prior to a certain version will have been fully purged from the system. We'll also use it in conjunction with PurgeOutdatedReplicas below. - The `PurgeOutdatedReplicas` RPC. This too 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. 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. The `PurgeOutdatedReplicas` lets the migration manager do exactly as the name suggests, ensuring all "outdated" replicas are processed before declaring the specific cluster version bump complete. - The concept of a "replica state version". This is what's used to construct the migration manager's view of what's "outdated", telling us which migrations can be assumed to have run against a particular replica. When we introduce backwards incompatible changes to the replica state (for example using the unreplicated truncated state instead of the replicated variant), the version would inform us if, for a given replica, we should expect a state representation prior to, or after the migration (in our example this corresponds to whether or not we can assume an unreplicated truncated state). As part of this commit, we also re-order the steps taken by the migration manager so that it executes a given migration first before bumping version gates cluster wide. This is because we want authors of migrations to ascertain that their own migrations have run to completion, instead of attaching that check to the next version. --- This PR motivates all of the above by also onboarding the TruncatedAndRangeAppliedState migration, lets us do the following: i. Use the RangeAppliedState on all ranges ii. Use the unreplicated TruncatedState on all ranges In 21.2 we'll finally be able to delete holdover code that knows how to handle the legacy replicated truncated state. Release note (general change): Cluster version upgrades, as initiated by SET CLUSTER SETTING version = -, now perform internal maintenance duties that will delay how long it takes for the command to complete. The delay is proportional to the amount of data currently stored in the cluster. The cluster will also experience a small amount of additional load during this period while the upgrade is being finalized. --- The ideas here follow from our original prototype in #57445. --- docs/generated/settings/settings.html | 2 +- pkg/clusterversion/cockroach_versions.go | 28 + pkg/clusterversion/key_string.go | 7 +- pkg/keys/constants.go | 2 + pkg/keys/doc.go | 1 + pkg/keys/keys.go | 10 + pkg/keys/printer.go | 1 + pkg/keys/printer_test.go | 1 + pkg/kv/batch.go | 37 +- pkg/kv/db.go | 10 + pkg/kv/kvserver/BUILD.bazel | 1 + pkg/kv/kvserver/batcheval/BUILD.bazel | 1 + .../kvserver/batcheval/cmd_end_transaction.go | 17 +- pkg/kv/kvserver/batcheval/cmd_migrate.go | 147 ++ pkg/kv/kvserver/batcheval/result/result.go | 7 + pkg/kv/kvserver/client_migration_test.go | 299 +++ pkg/kv/kvserver/client_replica_test.go | 75 + pkg/kv/kvserver/client_test.go | 18 +- pkg/kv/kvserver/debug_print.go | 3 + pkg/kv/kvserver/kvserverpb/state.pb.go | 259 +- pkg/kv/kvserver/kvserverpb/state.proto | 31 + pkg/kv/kvserver/replica.go | 7 + pkg/kv/kvserver/replica_application_result.go | 9 + .../replica_application_state_machine.go | 16 + pkg/kv/kvserver/replica_proposal.go | 11 + pkg/kv/kvserver/replica_raft.go | 5 + .../kvserver/replica_raft_truncation_test.go | 2 +- pkg/kv/kvserver/replica_test.go | 5 +- pkg/kv/kvserver/replica_write.go | 53 + pkg/kv/kvserver/stateloader/initial.go | 43 +- pkg/kv/kvserver/stateloader/stateloader.go | 39 +- pkg/kv/kvserver/store.go | 53 + pkg/kv/kvserver/store_create_replica.go | 1 + pkg/kv/kvserver/store_init.go | 13 +- pkg/kv/kvserver/store_raft.go | 6 +- pkg/kv/kvserver/store_snapshot.go | 7 + pkg/kv/kvserver/store_test.go | 10 +- pkg/kv/kvserver/testing_knobs.go | 7 + pkg/migration/BUILD.bazel | 9 + pkg/migration/helper.go | 134 +- pkg/migration/helper_test.go | 72 +- pkg/migration/manager.go | 119 +- pkg/migration/migrations.go | 113 +- pkg/migration/migrations_test.go | 225 ++ 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/server/init.go | 5 + pkg/server/migration.go | 46 + pkg/server/migration_test.go | 34 + pkg/server/node.go | 6 +- pkg/server/serverpb/migration.pb.go | 624 ++++- pkg/server/serverpb/migration.proto | 30 +- .../localtestcluster/local_test_cluster.go | 1 + pkg/ts/catalog/chart_catalog.go | 1 + 58 files changed, 3872 insertions(+), 1169 deletions(-) create mode 100644 pkg/kv/kvserver/batcheval/cmd_migrate.go create mode 100644 pkg/kv/kvserver/client_migration_test.go create mode 100644 pkg/migration/migrations_test.go diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 072c62542d4d..f044a7d0a9a3 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -100,6 +100,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen at https:///debug/requests trace.lightstep.tokenstringif set, traces go to Lightstep using this token trace.zipkin.collectorstringif set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set -versionversion20.2-10set the active cluster version in the format '.' +versionversion20.2-16set the active cluster version in the format '.' diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 660673d9fa31..fa79cb4e6dea 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -206,6 +206,22 @@ const ( VirtualComputedColumns // CPutInline is conditional put support for inline values. CPutInline + // ReplicaVersions enables the versioning of Replica state. + ReplicaVersions + // TruncatedAndRangeAppliedStateMigration is part of the migration to stop + // using the legacy truncated state within KV. After the migration, we'll be + // using the unreplicated truncated state and the RangeAppliedState on all + // ranges. Callers that wish to assert on there no longer being any legacy + // will be able to do so after PostTruncatedAndRangeAppliedStateMigration is + // active. This lets remove any holdover code handling the possibility of + // replicated truncated state in 21.2. + // + // TODO(irfansharif): Do the above in 21.2. + TruncatedAndRangeAppliedStateMigration + // PostTruncatedAndRangeAppliedStateMigration is used to purge all replicas + // using the replicated legacy TruncatedState. It's also used in asserting + // that no replicated truncated state representation is found. + PostTruncatedAndRangeAppliedStateMigration // Step (1): Add new versions here. ) @@ -340,6 +356,18 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: CPutInline, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 10}, }, + { + Key: ReplicaVersions, + Version: roachpb.Version{Major: 20, Minor: 2, Internal: 12}, + }, + { + Key: TruncatedAndRangeAppliedStateMigration, + Version: roachpb.Version{Major: 20, Minor: 2, Internal: 14}, + }, + { + Key: PostTruncatedAndRangeAppliedStateMigration, + Version: roachpb.Version{Major: 20, Minor: 2, Internal: 16}, + }, // Step (2): Add new versions here. }) diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index c65e5e2d8ae4..397d417996fd 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -35,11 +35,14 @@ func _() { _ = x[UniqueWithoutIndexConstraints-24] _ = x[VirtualComputedColumns-25] _ = x[CPutInline-26] + _ = x[ReplicaVersions-27] + _ = x[TruncatedAndRangeAppliedStateMigration-28] + _ = x[PostTruncatedAndRangeAppliedStateMigration-29] } -const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesClientRangeInfosOnBatchResponseNodeMembershipStatusRangeStatsRespHasDescMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInline" +const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesClientRangeInfosOnBatchResponseNodeMembershipStatusRangeStatsRespHasDescMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesUniqueWithoutIndexConstraintsVirtualComputedColumnsCPutInlineReplicaVersionsTruncatedAndRangeAppliedStateMigrationPostTruncatedAndRangeAppliedStateMigration" -var _Key_index = [...]uint16{0, 25, 34, 48, 53, 68, 90, 124, 145, 163, 180, 211, 231, 252, 269, 283, 347, 364, 373, 398, 418, 430, 435, 444, 472, 501, 523, 533} +var _Key_index = [...]uint16{0, 25, 34, 48, 53, 68, 90, 124, 145, 163, 180, 211, 231, 252, 269, 283, 347, 364, 373, 398, 418, 430, 435, 444, 472, 501, 523, 533, 548, 586, 628} func (i Key) String() string { if i < 0 || i >= Key(len(_Key_index)-1) { diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index 775ba0708d98..10f00d4a3788 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -93,6 +93,8 @@ var ( // LocalLeaseAppliedIndexLegacySuffix is the suffix for the applied lease // index. LocalLeaseAppliedIndexLegacySuffix = []byte("rlla") + // LocalRangeVersionSuffix is the suffix for the range version. + LocalRangeVersionSuffix = []byte("rver") // LocalRangeStatsLegacySuffix is the suffix for range statistics. LocalRangeStatsLegacySuffix = []byte("stat") // localTxnSpanGCThresholdSuffix is DEPRECATED and remains to prevent reuse. diff --git a/pkg/keys/doc.go b/pkg/keys/doc.go index a40102b82908..1abc504b8bff 100644 --- a/pkg/keys/doc.go +++ b/pkg/keys/doc.go @@ -189,6 +189,7 @@ var _ = [...]interface{}{ RaftTruncatedStateLegacyKey, // "rftt" RangeLeaseKey, // "rll-" LeaseAppliedIndexLegacyKey, // "rlla" + RangeVersionKey, // "rver" RangeStatsLegacyKey, // "stat" // 2. Unreplicated range-ID local keys: These contain metadata that diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index bc5cab2699d9..2b0bf24a2c3c 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -286,6 +286,11 @@ func RangeLastGCKey(rangeID roachpb.RangeID) roachpb.Key { return MakeRangeIDPrefixBuf(rangeID).RangeLastGCKey() } +// RangeVersionKey returns a system-local for the range version. +func RangeVersionKey(rangeID roachpb.RangeID) roachpb.Key { + return MakeRangeIDPrefixBuf(rangeID).RangeVersionKey() +} + // MakeRangeIDUnreplicatedPrefix creates a range-local key prefix from // rangeID for all unreplicated data. func MakeRangeIDUnreplicatedPrefix(rangeID roachpb.RangeID) roachpb.Key { @@ -964,6 +969,11 @@ func (b RangeIDPrefixBuf) RangeLastGCKey() roachpb.Key { return append(b.replicatedPrefix(), LocalRangeLastGCSuffix...) } +// RangeVersionKey returns a system-local key for the range version. +func (b RangeIDPrefixBuf) RangeVersionKey() roachpb.Key { + return append(b.replicatedPrefix(), LocalRangeVersionSuffix...) +} + // RangeTombstoneKey returns a system-local key for a range tombstone. func (b RangeIDPrefixBuf) RangeTombstoneKey() roachpb.Key { return append(b.unreplicatedPrefix(), LocalRangeTombstoneSuffix...) diff --git a/pkg/keys/printer.go b/pkg/keys/printer.go index 36cfda1b7ffb..e3a03bdcd7ff 100644 --- a/pkg/keys/printer.go +++ b/pkg/keys/printer.go @@ -171,6 +171,7 @@ var ( {name: "RangeLease", suffix: LocalRangeLeaseSuffix}, {name: "RangeStats", suffix: LocalRangeStatsLegacySuffix}, {name: "RangeLastGC", suffix: LocalRangeLastGCSuffix}, + {name: "RangeVersion", suffix: LocalRangeVersionSuffix}, } rangeSuffixDict = []struct { diff --git a/pkg/keys/printer_test.go b/pkg/keys/printer_test.go index 679720c559fa..655b857f0f8b 100644 --- a/pkg/keys/printer_test.go +++ b/pkg/keys/printer_test.go @@ -76,6 +76,7 @@ func TestPrettyPrint(t *testing.T) { {keys.RangeLeaseKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeLease", revertSupportUnknown}, {keys.RangeStatsLegacyKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeStats", revertSupportUnknown}, {keys.RangeLastGCKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeLastGC", revertSupportUnknown}, + {keys.RangeVersionKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/r/RangeVersion", revertSupportUnknown}, {keys.RaftHardStateKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/u/RaftHardState", revertSupportUnknown}, {keys.RangeTombstoneKey(roachpb.RangeID(1000001)), "/Local/RangeID/1000001/u/RangeTombstone", revertSupportUnknown}, diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index 46ede616d087..f89954b79eac 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -228,18 +228,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: @@ -265,6 +257,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 { @@ -786,3 +784,26 @@ func (b *Batch) addSSTable( b.appendReqs(req) b.initResult(1, 0, notRaw, nil) } + +// migrate is only exported on DB. +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 2aa1e37fa3d8..a0d193fc5ae1 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -681,6 +681,16 @@ func (db *DB) AddSSTable( return getOneErr(db.Run(ctx, b), b) } +// Migrate is used instruct all ranges overlapping with the provided keyspace 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. +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/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index f45629a8ec84..f69d024ab81a 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -204,6 +204,7 @@ go_test( "client_lease_test.go", "client_merge_test.go", "client_metrics_test.go", + "client_migration_test.go", "client_protectedts_test.go", "client_raft_helpers_test.go", "client_raft_log_queue_test.go", diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel index f163768f9a84..74533bf6b386 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_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 4e8b3d5f847a..9571d9284d2f 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -901,7 +901,7 @@ func splitTriggerHelper( // initial state. Additionally, since bothDeltaMS is tracking writes to // both sides, we need to update it as well. { - // Various pieces of code rely on a replica's lease never being unitialized, + // Various pieces of code rely on a replica's lease never being uninitialized, // but it's more than that - it ensures that we properly initialize the // timestamp cache, which is only populated on the lease holder, from that // of the original Range. We found out about a regression here the hard way @@ -918,8 +918,9 @@ func splitTriggerHelper( // - node two can illegally propose a write to 'd' at a lower timestamp. // // TODO(tschottdorf): why would this use r.store.Engine() and not the - // batch? - leftLease, err := MakeStateLoader(rec).LoadLease(ctx, rec.Engine()) + // batch? We do the same thing for other usages of the state loader. + sl := MakeStateLoader(rec) + leftLease, err := sl.LoadLease(ctx, rec.Engine()) if err != nil { return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to load lease") } @@ -936,8 +937,7 @@ func splitTriggerHelper( } rightLease := leftLease rightLease.Replica = replica - - gcThreshold, err := MakeStateLoader(rec).LoadGCThreshold(ctx, rec.Engine()) + gcThreshold, err := sl.LoadGCThreshold(ctx, rec.Engine()) if err != nil { return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to load GCThreshold") } @@ -968,6 +968,11 @@ func splitTriggerHelper( truncStateType = stateloader.TruncatedStateLegacyReplicated } + replicaVersion, err := sl.LoadVersion(ctx, rec.Engine()) + if err != nil { + return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to load GCThreshold") + } + // Writing the initial state is subtle since this also seeds the Raft // group. It becomes more subtle due to proposer-evaluated Raft. // @@ -1000,7 +1005,7 @@ func splitTriggerHelper( *h.AbsPostSplitRight(), err = stateloader.WriteInitialReplicaState( ctx, batch, *h.AbsPostSplitRight(), split.RightDesc, rightLease, - *gcThreshold, truncStateType, + *gcThreshold, truncStateType, replicaVersion, ) if err != nil { return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to write initial Replica state") diff --git a/pkg/kv/kvserver/batcheval/cmd_migrate.go b/pkg/kv/kvserver/batcheval/cmd_migrate.go new file mode 100644 index 000000000000..c0bea50ef924 --- /dev/null +++ b/pkg/kv/kvserver/batcheval/cmd_migrate.go @@ -0,0 +1,147 @@ +// 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/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval/result" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/errors" +) + +func init() { + RegisterReadWriteCommand(roachpb.Migrate, declareKeysMigrate, Migrate) +} + +func declareKeysMigrate( + desc *roachpb.RangeDescriptor, + header roachpb.Header, + _ roachpb.Request, + latchSpans, lockSpans *spanset.SpanSet, +) { + // TODO(irfansharif): This will eventually grow to capture the super set of + // all keys accessed by all migrations defined here. That could get + // cumbersome. We could spruce up the migration type and allow authors to + // define the allow authors for specific set of keys each migration needs to + // grab latches and locks over. + + latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeVersionKey(header.RangeID)}) + latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(desc.StartKey)}) + latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RaftTruncatedStateLegacyKey(header.RangeID)}) + lockSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RaftTruncatedStateLegacyKey(header.RangeID)}) +} + +// 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.TruncatedAndRangeAppliedStateMigration, truncatedAndAppliedStateMigration) +} + +func registerMigration(key clusterversion.Key, migration migration) { + migrationRegistry[clusterversion.ByKey(key)] = migration +} + +// Migrate executes the below-raft migration corresponding to the given version. +// See roachpb.MigrateRequest for more details. +func Migrate( + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, _ roachpb.Response, +) (result.Result, error) { + args := cArgs.Args.(*roachpb.MigrateRequest) + migrationVersion := args.Version + + fn, ok := migrationRegistry[migrationVersion] + if !ok { + return result.Result{}, errors.Newf("migration for %s not found", migrationVersion) + } + pd, err := fn(ctx, readWriter, cArgs) + if err != nil { + return result.Result{}, err + } + + // Since we're a below raft migration, we'll need update our replica state + // version. + if err := MakeStateLoader(cArgs.EvalCtx).SetVersion( + ctx, readWriter, cArgs.Stats, &migrationVersion, + ); err != nil { + return result.Result{}, err + } + if pd.Replicated.State == nil { + pd.Replicated.State = &kvserverpb.ReplicaState{} + } + // NB: We don't check for clusterversion.ReplicaVersions being active here + // as all below-raft migrations (the only users of Migrate) were introduced + // after it. + pd.Replicated.State.Version = &migrationVersion + return pd, nil +} + +// truncatedAndRangeAppliedStateMigration lets us stop using the legacy +// replicated truncated state and start using the new RangeAppliedState for this +// specific range. +func truncatedAndAppliedStateMigration( + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, +) (result.Result, error) { + var legacyTruncatedState roachpb.RaftTruncatedState + legacyKeyFound, err := storage.MVCCGetProto( + ctx, readWriter, keys.RaftTruncatedStateLegacyKey(cArgs.EvalCtx.GetRangeID()), + hlc.Timestamp{}, &legacyTruncatedState, storage.MVCCGetOptions{}, + ) + if err != nil { + return result.Result{}, err + } + + var pd result.Result + if legacyKeyFound { + // Time to migrate by deleting the legacy key. The downstream-of-Raft + // code will atomically rewrite the truncated state (supplied via the + // side effect) into the new unreplicated key. + if err := storage.MVCCDelete( + ctx, readWriter, cArgs.Stats, keys.RaftTruncatedStateLegacyKey(cArgs.EvalCtx.GetRangeID()), + hlc.Timestamp{}, nil, /* txn */ + ); err != nil { + return result.Result{}, err + } + pd.Replicated.State = &kvserverpb.ReplicaState{ + // We need to pass in a truncated state to enable the migration. + // Passing the same one is the easiest thing to do. + TruncatedState: &legacyTruncatedState, + } + } + return pd, nil +} + +// TestingRegisterMigrationInterceptor is used in tests to register an +// interceptor for a below-raft migration. +// +// TODO(irfansharif): This is a gross anti-pattern, we're letting tests mutate +// global state. This should instead be accessed EvalKnobs() instead. +func TestingRegisterMigrationInterceptor(version roachpb.Version, fn func()) (unregister func()) { + if _, ok := migrationRegistry[version]; ok { + panic("doubly registering migration") + } + migrationRegistry[version] = func(context.Context, storage.ReadWriter, CommandArgs) (result.Result, error) { + fn() + return result.Result{}, nil + } + return func() { delete(migrationRegistry, version) } +} diff --git a/pkg/kv/kvserver/batcheval/result/result.go b/pkg/kv/kvserver/batcheval/result/result.go index 788283865ac0..32271101021c 100644 --- a/pkg/kv/kvserver/batcheval/result/result.go +++ b/pkg/kv/kvserver/batcheval/result/result.go @@ -228,6 +228,13 @@ func (p *Result) MergeAndDestroy(q Result) error { q.Replicated.State.GCThreshold = nil } + if p.Replicated.State.Version == nil { + p.Replicated.State.Version = q.Replicated.State.Version + } else if q.Replicated.State.Version != nil { + return errors.AssertionFailedf("conflicting Version") + } + q.Replicated.State.Version = nil + if q.Replicated.State.Stats != nil { return errors.AssertionFailedf("must not specify Stats") } diff --git a/pkg/kv/kvserver/client_migration_test.go b/pkg/kv/kvserver/client_migration_test.go new file mode 100644 index 000000000000..2796e18c4b40 --- /dev/null +++ b/pkg/kv/kvserver/client_migration_test.go @@ -0,0 +1,299 @@ +// 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 kvserver_test + +import ( + "context" + "strings" + "sync" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +// TestStorePurgeOutdatedReplicas sets up a replica in the replica GC +// queue without actually processing it, runs a migration against all other +// replicas of the range (thus rendering the GC-able replica as "outdated", i.e. +// with a replica version less than the latest possible). It then checks to see +// that PurgeOutdatedReplicas does in fact remove the outdated replica. +func TestStorePurgeOutdatedReplicas(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const numStores = 3 + + ctx := context.Background() + migrationVersion := roachpb.Version{Major: 42} + tc := testcluster.StartTestCluster(t, numStores, + base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + DisableEagerReplicaRemoval: true, + DisableReplicaGCQueue: true, + }, + }, + }, + }, + ) + defer tc.Stopper().Stop(context.Background()) + + // Create our scratch range and replicate it to n2 and n3. + n1, n2, n3 := 0, 1, 2 + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Target(n2), tc.Target(n3)) + require.NoError(t, tc.WaitForVoters(k, tc.Target(n2), tc.Target(n3))) + + for _, node := range []int{n2, n3} { + ts := tc.Servers[node] + store, pErr := ts.Stores().GetStore(ts.GetFirstStoreID()) + if pErr != nil { + t.Fatal(pErr) + } + + require.NotNil(t, store.LookupReplica(roachpb.RKey(k))) + } + + // Mark the replica on n2 as eligible for GC. + desc := tc.RemoveVotersOrFatal(t, k, tc.Target(n2)) + + // We register an interceptor seeing as how we're attempting a (dummy) below + // raft migration below. + unregister := batcheval.TestingRegisterMigrationInterceptor(migrationVersion, func() {}) + defer unregister() + + // Migrate the remaining replicas on n1 and n3. + if err := tc.Server(n1).DB().Migrate(ctx, desc.StartKey, desc.EndKey, migrationVersion); err != nil { + t.Fatal(err) + } + + ts := tc.Servers[n2] + store, pErr := ts.Stores().GetStore(ts.GetFirstStoreID()) + if pErr != nil { + t.Fatal(pErr) + } + + // Check to see that the replica still exists on n2. + require.NotNil(t, store.LookupReplica(roachpb.RKey(k))) + + if err := store.PurgeOutdatedReplicas(ctx, migrationVersion); err != nil { + t.Fatal(err) + } + + // Check to see that the replica was purged from n2. + require.Nil(t, store.LookupReplica(roachpb.RKey(k))) +} + +// TestMigrateWithInflightSnapshot checks to see that the Migrate command blocks +// in the face of an in-flight snapshot that hasn't yet instantiated the +// target replica. We expect the Migrate command to wait for its own application +// on all replicas, including learners. +func TestMigrateWithInflightSnapshot(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + var once sync.Once + blockUntilSnapshotCh := make(chan struct{}) + blockSnapshotsCh := make(chan struct{}) + knobs, ltk := makeReplicationTestKnobs() + ltk.storeKnobs.DisableRaftSnapshotQueue = true // we'll control it ourselves + ltk.storeKnobs.ReceiveSnapshot = func(h *kvserver.SnapshotRequest_Header) error { + // We'll want a signal for when the snapshot was received by the sender. + once.Do(func() { close(blockUntilSnapshotCh) }) + + // We'll also want to temporarily stall incoming snapshots. + select { + case <-blockSnapshotsCh: + case <-time.After(10 * time.Second): + return errors.New(`test timed out`) + } + return nil + } + ctx := context.Background() + migrationVersion := roachpb.Version{Major: 42} + tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{Knobs: knobs}, + ReplicationMode: base.ReplicationManual, + }) + defer tc.Stopper().Stop(ctx) + + k := tc.ScratchRange(t) + g := ctxgroup.WithContext(ctx) + n1, n2 := 0, 1 + g.GoCtx(func(ctx context.Context) error { + _, err := tc.AddVoters(k, tc.Target(n2)) + return err + }) + + // Wait until the snapshot starts, which happens after the learner has been + // added. + <-blockUntilSnapshotCh + desc := tc.LookupRangeOrFatal(t, k) + require.Len(t, desc.Replicas().Voters(), 1) + require.Len(t, desc.Replicas().Learners(), 1) + + // Enqueue the replica in the raftsnapshot queue. We use SucceedsSoon + // because it may take a bit for raft to figure out that we need to be + // generating a snapshot. + store := tc.GetFirstStoreFromServer(t, 0) + repl, err := store.GetReplica(desc.RangeID) + require.NoError(t, err) + testutils.SucceedsSoon(t, func() error { + trace, processErr, err := store.ManuallyEnqueue(ctx, "raftsnapshot", repl, true /* skipShouldQueue */) + if err != nil { + return err + } + if processErr != nil { + return processErr + } + const msg = `skipping snapshot; replica is likely a learner in the process of being added: (n2,s2):2LEARNER` + formattedTrace := trace.String() + if !strings.Contains(formattedTrace, msg) { + return errors.Errorf(`expected "%s" in trace got:\n%s`, msg, formattedTrace) + } + return nil + }) + + unregister := batcheval.TestingRegisterMigrationInterceptor(migrationVersion, func() {}) + defer unregister() + + // Migrate the scratch range. We expect this to hang given the in-flight + // snapshot is held up. + func() { + cCtx, cancel := context.WithTimeout(ctx, 50*time.Millisecond) + defer cancel() + + err := tc.Server(n1).DB().Migrate(cCtx, desc.StartKey, desc.EndKey, migrationVersion) + require.True(t, testutils.IsError(err, context.DeadlineExceeded.Error()), err) + }() + + // Unblock the snapshot and let the learner get promoted to a voter. + close(blockSnapshotsCh) + require.NoError(t, g.Wait()) + + // We expect the migration attempt to go through now. + if err := tc.Server(n1).DB().Migrate(ctx, desc.StartKey, desc.EndKey, migrationVersion); err != nil { + t.Fatal(err) + } + + for _, node := range []int{n1, n2} { + ts := tc.Servers[node] + store, pErr := ts.Stores().GetStore(ts.GetFirstStoreID()) + if pErr != nil { + t.Fatal(pErr) + } + + repl := store.LookupReplica(roachpb.RKey(k)) + require.NotNil(t, repl) + require.Equal(t, repl.Version(), migrationVersion) + } +} + +// TestMigrateWaitsForApplication checks to see that migrate commands wait to be +// applied on all replicas before returning to the caller. +func TestMigrateWaitsForApplication(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + n1, n2, n3 := 0, 1, 2 + blockApplicationCh := make(chan struct{}) + + // We're going to be migrating from startV to endV. + startV := roachpb.Version{Major: 41} + endV := roachpb.Version{Major: 42} + + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: cluster.MakeTestingClusterSettingsWithVersions(endV, startV, false), + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + BinaryVersionOverride: startV, + DisableAutomaticVersionUpgrade: 1, + }, + Store: &kvserver.StoreTestingKnobs{ + TestingApplyFilter: func(args kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + if args.StoreID == roachpb.StoreID(n3) && args.State != nil && args.State.Version != nil { + <-blockApplicationCh + } + return 0, nil + }, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + // Create our scratch range and replicate it to n2 and n3. + k := tc.ScratchRange(t) + tc.AddVotersOrFatal(t, k, tc.Target(n2), tc.Target(n3)) + require.NoError(t, tc.WaitForVoters(k, tc.Target(n2), tc.Target(n3))) + + for _, node := range []int{n1, n2, n3} { + ts := tc.Servers[node] + store, pErr := ts.Stores().GetStore(ts.GetFirstStoreID()) + if pErr != nil { + t.Fatal(pErr) + } + + repl := store.LookupReplica(roachpb.RKey(k)) + require.NotNil(t, repl) + require.Equal(t, repl.Version(), startV) + } + + desc := tc.LookupRangeOrFatal(t, k) + unregister := batcheval.TestingRegisterMigrationInterceptor(endV, func() {}) + defer unregister() + + // Migrate the scratch range. We expect this to hang given we've gated the + // command application on n3. + func() { + cCtx, cancel := context.WithTimeout(ctx, 50*time.Millisecond) + defer cancel() + + err := tc.Server(n1).DB().Migrate(cCtx, desc.StartKey, desc.EndKey, endV) + require.True(t, testutils.IsError(err, context.DeadlineExceeded.Error()), err) + }() + + close(blockApplicationCh) + + // We expect the migration attempt to go through now. + if err := tc.Server(n1).DB().Migrate(ctx, desc.StartKey, desc.EndKey, endV); err != nil { + t.Fatal(err) + } + + for _, node := range []int{n1, n2, n3} { + ts := tc.Servers[node] + store, pErr := ts.Stores().GetStore(ts.GetFirstStoreID()) + if pErr != nil { + t.Fatal(pErr) + } + + repl := store.LookupReplica(roachpb.RKey(k)) + require.NotNil(t, repl) + require.Equal(t, repl.Version(), endV) + } +} diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index c62813bccbcd..8fd5fd9907b8 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -33,8 +33,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" @@ -3576,6 +3578,79 @@ func TestTenantID(t *testing.T) { } +// TestRangeMigration tests the below-raft migration infrastructure. It checks +// to see that the version recorded as part of the in-memory ReplicaState +// is up to date, and in-sync with the persisted state. It also checks to see +// that the right registered migration is invoked. +func TestRangeMigration(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // We're going to be transitioning from startV to endV. Think a cluster of + // binaries running vX, but with active version vX-1. + startV := roachpb.Version{Major: 41} + endV := roachpb.Version{Major: 42} + + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: cluster.MakeTestingClusterSettingsWithVersions(endV, startV, false), + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + BinaryVersionOverride: startV, + DisableAutomaticVersionUpgrade: 1, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + key := tc.ScratchRange(t) + desc, err := tc.LookupRange(key) + require.NoError(t, err) + rangeID := desc.RangeID + + store := tc.GetFirstStoreFromServer(t, 0) + assertVersion := func(expV roachpb.Version) { + repl, err := store.GetReplica(rangeID) + if err != nil { + t.Fatal(err) + } + if gotV := repl.Version(); gotV != expV { + t.Fatalf("expected in-memory version %s, got %s", expV, gotV) + } + + sl := stateloader.Make(rangeID) + persistedV, err := sl.LoadVersion(ctx, store.Engine()) + if err != nil { + t.Fatal(err) + } + if persistedV != expV { + t.Fatalf("expected persisted version %s, got %s", expV, persistedV) + } + } + + assertVersion(startV) + + migrated := false + unregister := batcheval.TestingRegisterMigrationInterceptor(endV, func() { + migrated = true + }) + defer unregister() + + kvDB := tc.Servers[0].DB() + req := migrateArgs(desc.StartKey.AsRawKey(), desc.EndKey.AsRawKey(), endV) + if _, pErr := kv.SendWrappedWith(ctx, kvDB.GetFactory().NonTransactionalSender(), roachpb.Header{RangeID: desc.RangeID}, req); pErr != nil { + t.Fatal(pErr) + } + + if !migrated { + t.Fatalf("expected migration interceptor to have been called") + } + assertVersion(endV) +} + func TestRaftSchedulerPrioritizesNodeLiveness(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/kv/kvserver/client_test.go b/pkg/kv/kvserver/client_test.go index 4cb5b186b93e..06b14a6d79c4 100644 --- a/pkg/kv/kvserver/client_test.go +++ b/pkg/kv/kvserver/client_test.go @@ -213,7 +213,9 @@ func createTestStoreWithOpts( eng, kvs, /* initialValues */ clusterversion.TestingBinaryVersion, - 1 /* numStores */, splits, storeCfg.Clock.PhysicalNow()) + 1 /* numStores */, splits, storeCfg.Clock.PhysicalNow(), + storeCfg.TestingKnobs, + ) if err != nil { t.Fatal(err) } @@ -959,7 +961,9 @@ func (m *multiTestContext) addStore(idx int) { eng, kvs, /* initialValues */ clusterversion.TestingBinaryVersion, - len(m.engines), splits, cfg.Clock.PhysicalNow()) + len(m.engines), splits, cfg.Clock.PhysicalNow(), + cfg.TestingKnobs, + ) if err != nil { m.t.Fatal(err) } @@ -1572,6 +1576,16 @@ func pushTxnArgs( } } +func migrateArgs(start, end roachpb.Key, version roachpb.Version) *roachpb.MigrateRequest { + return &roachpb.MigrateRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: start, + EndKey: end, + }, + Version: version, + } +} + func adminTransferLeaseArgs(key roachpb.Key, target roachpb.StoreID) roachpb.Request { return &roachpb.AdminTransferLeaseRequest{ RequestHeader: roachpb.RequestHeader{ diff --git a/pkg/kv/kvserver/debug_print.go b/pkg/kv/kvserver/debug_print.go index 077d7af25caa..bca80f657984 100644 --- a/pkg/kv/kvserver/debug_print.go +++ b/pkg/kv/kvserver/debug_print.go @@ -269,6 +269,9 @@ func tryRangeIDKey(kv storage.MVCCKeyValue) (string, error) { case bytes.Equal(suffix, keys.LocalRangeLastGCSuffix): msg = &hlc.Timestamp{} + case bytes.Equal(suffix, keys.LocalRangeVersionSuffix): + msg = &roachpb.Version{} + case bytes.Equal(suffix, keys.LocalRangeTombstoneSuffix): msg = &roachpb.RangeTombstone{} diff --git a/pkg/kv/kvserver/kvserverpb/state.pb.go b/pkg/kv/kvserver/kvserverpb/state.pb.go index 46d4982b269e..7429d5db4289 100644 --- a/pkg/kv/kvserver/kvserverpb/state.pb.go +++ b/pkg/kv/kvserver/kvserverpb/state.pb.go @@ -71,13 +71,44 @@ type ReplicaState struct { // is idempotent by Replica state machines, meaning that it is ok for multiple // Raft commands to set it to true. UsingAppliedStateKey bool `protobuf:"varint,11,opt,name=using_applied_state_key,json=usingAppliedStateKey,proto3" json:"using_applied_state_key,omitempty"` + // Version tells us which migrations can be assumed to have run against this + // particular replica. When we introduce backwards incompatible changes to the + // replica state (for example using the unreplicated truncated state instead + // of the replicated variant), this field tells us if we should expect a + // state representation prior to, or after the change (in our example it + // tells us whether or not we can assume an unreplicated truncated state). All + // migrations associated with versions less than or equal to this one, can be + // assumed to have run. + // + // We should note that the progression of cluster versions in the cluster does + // not happen in lock step with replica versions for all replicas in the + // system. Well, not for all cluster versions. Replica versions are one the + // primitives used to coordinate below-raft migrations, and when cluster + // versions for those are being rolled out cluster-wide, the expectation is + // that all replica versions are bumped along side it. For everything else, + // the coupling is neither necessary nor enforced. When a new replica is being + // initialized, it's annotated with a replica version. This happens in one of + // two places: + // - during cluster creation time, in which case we source the binary version + // (i.e. the version the cluster is bootstrapped with) + // - replicas created through snapshots or splits, where we inherit the + // version from the incoming snapshot or the LHS of the split respectively. + // + // While a below-raft migration is underway, replicas from different ranges + // may have differing replica versions. This is fine, we rely on higher level + // orchestration primitives[1][2] to shepherd all replicas in the system to + // the right version. + // + // [1]: migration.Manager + // [2]: PurgeOutdatedReplicas + Version *roachpb.Version `protobuf:"bytes,12,opt,name=version,proto3" json:"version,omitempty"` } func (m *ReplicaState) Reset() { *m = ReplicaState{} } func (m *ReplicaState) String() string { return proto.CompactTextString(m) } func (*ReplicaState) ProtoMessage() {} func (*ReplicaState) Descriptor() ([]byte, []int) { - return fileDescriptor_state_32aca65b437d2266, []int{0} + return fileDescriptor_state_50e6a6403c7eff35, []int{0} } func (m *ReplicaState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -149,7 +180,7 @@ func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (m *RangeInfo) String() string { return proto.CompactTextString(m) } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_state_32aca65b437d2266, []int{1} + return fileDescriptor_state_50e6a6403c7eff35, []int{1} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -185,7 +216,7 @@ func (m *RangeInfo_CTEntry) Reset() { *m = RangeInfo_CTEntry{} } func (m *RangeInfo_CTEntry) String() string { return proto.CompactTextString(m) } func (*RangeInfo_CTEntry) ProtoMessage() {} func (*RangeInfo_CTEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_state_32aca65b437d2266, []int{1, 0} + return fileDescriptor_state_50e6a6403c7eff35, []int{1, 0} } func (m *RangeInfo_CTEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -221,7 +252,7 @@ func (m *LatchManagerInfo) Reset() { *m = LatchManagerInfo{} } func (m *LatchManagerInfo) String() string { return proto.CompactTextString(m) } func (*LatchManagerInfo) ProtoMessage() {} func (*LatchManagerInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_state_32aca65b437d2266, []int{2} + return fileDescriptor_state_50e6a6403c7eff35, []int{2} } func (m *LatchManagerInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -295,6 +326,9 @@ func (this *ReplicaState) Equal(that interface{}) bool { if this.UsingAppliedStateKey != that1.UsingAppliedStateKey { return false } + if !this.Version.Equal(that1.Version) { + return false + } return true } func (this *RangeInfo) Equal(that interface{}) bool { @@ -486,6 +520,16 @@ func (m *ReplicaState) MarshalTo(dAtA []byte) (int, error) { } i++ } + if m.Version != nil { + dAtA[i] = 0x62 + i++ + i = encodeVarintState(dAtA, i, uint64(m.Version.Size())) + n6, err := m.Version.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n6 + } return i, nil } @@ -507,11 +551,11 @@ func (m *RangeInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintState(dAtA, i, uint64(m.ReplicaState.Size())) - n6, err := m.ReplicaState.MarshalTo(dAtA[i:]) + n7, err := m.ReplicaState.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n6 + i += n7 if m.LastIndex != 0 { dAtA[i] = 0x10 i++ @@ -555,19 +599,19 @@ func (m *RangeInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintState(dAtA, i, uint64(m.NewestClosedTimestamp.Size())) - n7, err := m.NewestClosedTimestamp.MarshalTo(dAtA[i:]) + n8, err := m.NewestClosedTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n7 + i += n8 dAtA[i] = 0x62 i++ i = encodeVarintState(dAtA, i, uint64(m.ActiveClosedTimestamp.Size())) - n8, err := m.ActiveClosedTimestamp.MarshalTo(dAtA[i:]) + n9, err := m.ActiveClosedTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n8 + i += n9 if m.RangefeedRegistrations != 0 { dAtA[i] = 0x68 i++ @@ -579,22 +623,22 @@ func (m *RangeInfo) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintState(dAtA, i, uint64(m.ProposalQuotaBaseIndex)) } if len(m.ProposalQuotaReleaseQueue) > 0 { - dAtA10 := make([]byte, len(m.ProposalQuotaReleaseQueue)*10) - var j9 int + dAtA11 := make([]byte, len(m.ProposalQuotaReleaseQueue)*10) + var j10 int for _, num1 := range m.ProposalQuotaReleaseQueue { num := uint64(num1) for num >= 1<<7 { - dAtA10[j9] = uint8(uint64(num)&0x7f | 0x80) + dAtA11[j10] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j9++ + j10++ } - dAtA10[j9] = uint8(num) - j9++ + dAtA11[j10] = uint8(num) + j10++ } dAtA[i] = 0x7a i++ - i = encodeVarintState(dAtA, i, uint64(j9)) - i += copy(dAtA[i:], dAtA10[:j9]) + i = encodeVarintState(dAtA, i, uint64(j10)) + i += copy(dAtA[i:], dAtA11[:j10]) } if m.TenantID != 0 { dAtA[i] = 0x80 @@ -637,11 +681,11 @@ func (m *RangeInfo_CTEntry) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintState(dAtA, i, uint64(m.ClosedTimestamp.Size())) - n11, err := m.ClosedTimestamp.MarshalTo(dAtA[i:]) + n12, err := m.ClosedTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n11 + i += n12 if m.MLAI != 0 { dAtA[i] = 0x18 i++ @@ -727,6 +771,10 @@ func (m *ReplicaState) Size() (n int) { if m.UsingAppliedStateKey { n += 2 } + if m.Version != nil { + l = m.Version.Size() + n += 1 + l + sovState(uint64(l)) + } return n } @@ -1086,6 +1134,39 @@ func (m *ReplicaState) Unmarshal(dAtA []byte) error { } } m.UsingAppliedStateKey = bool(v != 0) + case 12: + 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 ErrIntOverflowState + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthState + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Version == nil { + m.Version = &roachpb.Version{} + } + if err := m.Version.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipState(dAtA[iNdEx:]) @@ -1871,74 +1952,76 @@ var ( ) func init() { - proto.RegisterFile("kv/kvserver/kvserverpb/state.proto", fileDescriptor_state_32aca65b437d2266) + proto.RegisterFile("kv/kvserver/kvserverpb/state.proto", fileDescriptor_state_50e6a6403c7eff35) } -var fileDescriptor_state_32aca65b437d2266 = []byte{ - // 1040 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0x4f, 0x6f, 0xdb, 0xc6, - 0x13, 0x35, 0x23, 0xca, 0xa6, 0x56, 0x76, 0xac, 0xec, 0xcf, 0x89, 0x19, 0x27, 0x96, 0x0c, 0x01, - 0xbf, 0xc2, 0x05, 0x52, 0x12, 0x75, 0xd0, 0x16, 0xfd, 0x03, 0x14, 0x96, 0x5c, 0x14, 0x72, 0x6d, - 0x23, 0xde, 0x08, 0x3d, 0xa4, 0x07, 0x62, 0x45, 0x8e, 0x29, 0x42, 0x14, 0x97, 0x59, 0x2e, 0x15, - 0x3b, 0xd7, 0x7e, 0x81, 0x7e, 0x84, 0x7e, 0x1c, 0xa3, 0x27, 0x1f, 0x73, 0x12, 0x5a, 0xf9, 0xd2, - 0x5b, 0xef, 0x3d, 0x15, 0xbb, 0x4b, 0xc9, 0x92, 0x1d, 0x20, 0xb9, 0x2d, 0xdf, 0x9b, 0x19, 0x0e, - 0xe7, 0xbd, 0x1d, 0xa2, 0xe6, 0x60, 0xe4, 0x0e, 0x46, 0x19, 0xf0, 0x11, 0xf0, 0xd9, 0x21, 0xed, - 0xb9, 0x99, 0xa0, 0x02, 0x9c, 0x94, 0x33, 0xc1, 0x70, 0xc3, 0x67, 0xfe, 0x80, 0x33, 0xea, 0xf7, - 0x9d, 0xc1, 0xc8, 0x99, 0x06, 0x39, 0x99, 0x60, 0x9c, 0x86, 0x90, 0xf6, 0xb6, 0x9e, 0x14, 0x47, - 0x17, 0x92, 0x30, 0x4a, 0x20, 0xed, 0xb9, 0xc3, 0x91, 0xef, 0xeb, 0xec, 0xad, 0x27, 0x2a, 0x33, - 0xed, 0xb9, 0x51, 0x22, 0x80, 0x27, 0x34, 0xf6, 0x38, 0x3d, 0x13, 0x05, 0xf9, 0x68, 0x4a, 0x0e, - 0x41, 0xd0, 0x80, 0x0a, 0x5a, 0xe0, 0x78, 0x8a, 0xcf, 0x61, 0x76, 0x2e, 0xa2, 0xd8, 0xed, 0xc7, - 0xbe, 0x2b, 0xa2, 0x21, 0x64, 0x82, 0x0e, 0xd3, 0x82, 0xd9, 0x08, 0x59, 0xc8, 0xd4, 0xd1, 0x95, - 0x27, 0x8d, 0x36, 0x7f, 0x35, 0xd1, 0x2a, 0x81, 0x34, 0x8e, 0x7c, 0xfa, 0x52, 0x7e, 0x0d, 0x7e, - 0x86, 0xb0, 0x7c, 0xb5, 0x47, 0xd3, 0x34, 0x8e, 0x20, 0xf0, 0xa2, 0x24, 0x80, 0x73, 0xdb, 0xd8, - 0x31, 0x76, 0x4d, 0x52, 0x93, 0xcc, 0xbe, 0x26, 0x3a, 0x12, 0xc7, 0x0e, 0xfa, 0x5f, 0x0c, 0x34, - 0x83, 0x5b, 0xe1, 0xf7, 0x54, 0xf8, 0x03, 0x45, 0x2d, 0xc4, 0x7f, 0x89, 0xcc, 0x00, 0x32, 0xdf, - 0x2e, 0xed, 0x18, 0xbb, 0xd5, 0xbd, 0xa6, 0x73, 0x33, 0xb4, 0xe2, 0x5b, 0x1c, 0x42, 0x93, 0x10, - 0x0e, 0x20, 0xf3, 0x79, 0x94, 0x0a, 0xc6, 0x89, 0x8a, 0xc7, 0x0e, 0x2a, 0xab, 0x62, 0xb6, 0xa9, - 0x12, 0xed, 0xf7, 0x24, 0x1e, 0x49, 0x9e, 0xe8, 0x30, 0x7c, 0x82, 0xd6, 0x05, 0xcf, 0x13, 0x9f, - 0x0a, 0x08, 0x3c, 0x25, 0x93, 0x5d, 0x56, 0x99, 0xff, 0x7f, 0xef, 0x2b, 0xcf, 0x44, 0x77, 0x1a, - 0xad, 0xa6, 0x40, 0xee, 0x8b, 0x85, 0x67, 0x7c, 0x8a, 0x56, 0x43, 0xdf, 0x13, 0x7d, 0x0e, 0x59, - 0x9f, 0xc5, 0x81, 0xbd, 0xac, 0x8a, 0x6d, 0xcf, 0x15, 0x93, 0x73, 0x77, 0xfa, 0xb1, 0xef, 0x74, - 0xa7, 0x73, 0x6f, 0xad, 0x4f, 0xc6, 0x8d, 0xea, 0x8f, 0xed, 0xee, 0x34, 0x8b, 0x54, 0x43, 0x7f, - 0xf6, 0x80, 0xbf, 0x45, 0x65, 0xd9, 0x58, 0x66, 0xaf, 0xdc, 0x69, 0xac, 0x70, 0x8a, 0x33, 0x75, - 0x8a, 0x73, 0xfc, 0x73, 0xbb, 0x2d, 0x1b, 0xc9, 0x88, 0xce, 0xc1, 0x5f, 0xa0, 0xcd, 0x3c, 0x8b, - 0x92, 0x70, 0x36, 0x77, 0xf5, 0x8d, 0xde, 0x00, 0x2e, 0xec, 0xea, 0x8e, 0xb1, 0x6b, 0x91, 0x0d, - 0x45, 0x17, 0xb3, 0x57, 0xdf, 0xf0, 0x13, 0x5c, 0x7c, 0x63, 0xfe, 0xfd, 0x7b, 0xc3, 0x38, 0x34, - 0x2d, 0xab, 0x56, 0x39, 0x34, 0xad, 0x4a, 0x0d, 0x1d, 0x9a, 0x16, 0xaa, 0x55, 0x9b, 0x7f, 0x58, - 0xa8, 0xa2, 0x06, 0xdf, 0x49, 0xce, 0x18, 0x3e, 0xd6, 0x9d, 0x81, 0x52, 0xbd, 0xba, 0xf7, 0x99, - 0xf3, 0x01, 0x6b, 0x3b, 0xf3, 0x06, 0x6a, 0x59, 0x97, 0xe3, 0xc6, 0xd2, 0xd5, 0xb8, 0x61, 0xe8, - 0x5e, 0x01, 0x6f, 0x23, 0x14, 0xd3, 0x4c, 0x2c, 0x58, 0xa3, 0x22, 0x11, 0x6d, 0x89, 0x06, 0xaa, - 0x26, 0xf9, 0xd0, 0x4b, 0x21, 0x09, 0xa2, 0x24, 0x54, 0xce, 0x30, 0x09, 0x4a, 0xf2, 0xe1, 0x0b, - 0x8d, 0x4c, 0x03, 0x02, 0xce, 0xd2, 0x14, 0x02, 0xa5, 0xa3, 0x0e, 0x38, 0xd0, 0x08, 0x6e, 0xa2, - 0x35, 0x65, 0xd9, 0x98, 0x85, 0x5e, 0x16, 0xbd, 0x05, 0xa5, 0x4e, 0x89, 0x54, 0x25, 0x78, 0xc4, - 0xc2, 0x97, 0xd1, 0x5b, 0xc0, 0xdf, 0xa1, 0x2d, 0x9a, 0xa6, 0x9c, 0x9d, 0x47, 0x43, 0x39, 0xa8, - 0x94, 0xb3, 0x94, 0x65, 0x34, 0xf6, 0x5e, 0xe7, 0x4c, 0x50, 0x25, 0x41, 0x89, 0xd8, 0x73, 0x11, - 0x2f, 0x8a, 0x80, 0x53, 0xc9, 0xe3, 0x4f, 0xd0, 0x3a, 0x97, 0xe3, 0xf1, 0x86, 0xf4, 0xdc, 0xeb, - 0x5d, 0x08, 0xc8, 0x6c, 0x4b, 0xa5, 0xac, 0x29, 0xf8, 0x98, 0x9e, 0xb7, 0x24, 0x88, 0x3f, 0x47, - 0x0f, 0x17, 0x3a, 0xf1, 0x04, 0xcf, 0x33, 0x01, 0x81, 0x8d, 0x94, 0x28, 0x78, 0xae, 0xa3, 0xae, - 0x66, 0x70, 0x8a, 0x36, 0x13, 0x78, 0x03, 0x99, 0xf0, 0xfc, 0x98, 0x65, 0x10, 0x78, 0xb3, 0x7b, - 0xab, 0x94, 0xac, 0xee, 0xed, 0x7d, 0x78, 0xfc, 0x53, 0xe5, 0x9c, 0x76, 0xf7, 0x87, 0x44, 0xf0, - 0x8b, 0x96, 0x29, 0x35, 0x20, 0x0f, 0x75, 0xe1, 0xb6, 0xaa, 0x3b, 0xb3, 0x25, 0xfe, 0x05, 0x6d, - 0x52, 0x5f, 0x44, 0x23, 0xb8, 0xfb, 0xc6, 0xd5, 0x8f, 0xb1, 0x75, 0x51, 0x5c, 0xd7, 0xb8, 0x5d, - 0xfc, 0x2b, 0xb4, 0xa9, 0x46, 0x72, 0x06, 0x10, 0x78, 0x1c, 0xc2, 0x28, 0x13, 0x9c, 0x8a, 0x88, - 0x25, 0x99, 0xbd, 0xa6, 0x26, 0xf6, 0x68, 0x46, 0x93, 0x79, 0x16, 0x7f, 0x8d, 0x1e, 0x2f, 0x8a, - 0xe2, 0xf5, 0xe4, 0x5e, 0xd1, 0xa6, 0xb9, 0xaf, 0x53, 0xd3, 0x79, 0x51, 0x5a, 0x34, 0x03, 0xed, - 0xa0, 0xef, 0xd1, 0xd3, 0x5b, 0xa9, 0x1c, 0xf4, 0x56, 0x7a, 0x9d, 0x43, 0x0e, 0xf6, 0xfa, 0x4e, - 0x69, 0xb7, 0x44, 0x1e, 0x2f, 0x64, 0x13, 0x1d, 0x71, 0x2a, 0x03, 0xf0, 0xa7, 0xa8, 0x22, 0x20, - 0xa1, 0x89, 0xf0, 0xa2, 0xc0, 0xae, 0x49, 0x7f, 0xb5, 0x56, 0x27, 0xe3, 0x86, 0xd5, 0x55, 0x60, - 0xe7, 0x80, 0x58, 0x9a, 0xee, 0x04, 0xca, 0xcc, 0xcc, 0x1f, 0x78, 0x82, 0xf6, 0x62, 0xb0, 0x1f, - 0xec, 0x18, 0xbb, 0x15, 0x52, 0x91, 0x48, 0x57, 0x02, 0x5b, 0xff, 0x18, 0x68, 0xa5, 0x10, 0x01, - 0xbf, 0x42, 0x2b, 0x09, 0x0b, 0x40, 0xd6, 0x94, 0x17, 0xa9, 0xdc, 0xda, 0x9f, 0x8c, 0x1b, 0xcb, - 0x27, 0x2c, 0x80, 0xce, 0xc1, 0xbf, 0xe3, 0xc6, 0xf3, 0x30, 0x12, 0xfd, 0xbc, 0xe7, 0xf8, 0x6c, - 0xe8, 0xce, 0xe6, 0x1d, 0xf4, 0x6e, 0xce, 0x6e, 0x3a, 0x08, 0xdd, 0xe9, 0x8e, 0xd2, 0x69, 0x64, - 0x59, 0x56, 0xec, 0x04, 0xf8, 0x04, 0xd5, 0xee, 0x88, 0x77, 0xef, 0xe3, 0xc5, 0x5b, 0xf7, 0x6f, - 0xc9, 0xf6, 0x14, 0x99, 0xc3, 0x98, 0x46, 0xea, 0xf6, 0x95, 0x5a, 0xd6, 0x64, 0xdc, 0x30, 0x8f, - 0x8f, 0xf6, 0x3b, 0x44, 0xa1, 0x78, 0x03, 0x95, 0x21, 0x65, 0x7e, 0x5f, 0x6d, 0xdf, 0x12, 0xd1, - 0x0f, 0x7a, 0x99, 0xcc, 0x56, 0x8a, 0x59, 0x2b, 0xeb, 0x95, 0xd2, 0x24, 0xa8, 0x76, 0x44, 0x85, - 0xdf, 0x3f, 0xa6, 0x09, 0x0d, 0x81, 0xab, 0x95, 0xb2, 0x8d, 0x10, 0x07, 0x1a, 0x78, 0x3e, 0xcb, - 0x13, 0xa1, 0xc6, 0x51, 0x22, 0x15, 0x89, 0xb4, 0x25, 0x20, 0xaf, 0xf8, 0x1b, 0x1e, 0x09, 0x28, - 0xf8, 0x7b, 0x8a, 0x47, 0x0a, 0x52, 0x01, 0xad, 0x67, 0x97, 0x7f, 0xd5, 0x97, 0x2e, 0x27, 0x75, - 0xe3, 0x6a, 0x52, 0x37, 0xde, 0x4d, 0xea, 0xc6, 0x9f, 0x93, 0xba, 0xf1, 0xdb, 0x75, 0x7d, 0xe9, - 0xea, 0xba, 0xbe, 0xf4, 0xee, 0xba, 0xbe, 0xf4, 0x0a, 0xdd, 0xfc, 0x97, 0x7b, 0xcb, 0xea, 0xdf, - 0xf6, 0xfc, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x1b, 0x72, 0x76, 0x6f, 0xb8, 0x07, 0x00, 0x00, +var fileDescriptor_state_50e6a6403c7eff35 = []byte{ + // 1061 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0xcd, 0x6e, 0x1b, 0x37, + 0x10, 0xf6, 0x5a, 0x2b, 0x5b, 0xa2, 0xec, 0x58, 0x61, 0x9d, 0x78, 0xe3, 0xc4, 0x92, 0x21, 0xa0, + 0x85, 0x0b, 0xa4, 0x2b, 0xd4, 0xe9, 0x0f, 0xfa, 0x03, 0x14, 0x96, 0x5c, 0x14, 0x72, 0x6d, 0x23, + 0x66, 0x84, 0x1c, 0xd2, 0xc3, 0x82, 0xda, 0x1d, 0xaf, 0x16, 0x5a, 0x2d, 0x37, 0x24, 0xa5, 0xd8, + 0x79, 0x8a, 0x3e, 0x42, 0x5f, 0xa3, 0x6f, 0x60, 0xf4, 0xe4, 0x63, 0x4e, 0x42, 0x2b, 0x5f, 0x7a, + 0xeb, 0xbd, 0xa7, 0x80, 0xa4, 0x56, 0x96, 0x6c, 0x03, 0xc9, 0x8d, 0xfb, 0x7d, 0xdf, 0x0c, 0xc9, + 0xf9, 0x86, 0xb3, 0xa8, 0xd6, 0x1b, 0xd6, 0x7b, 0x43, 0x01, 0x7c, 0x08, 0x7c, 0xba, 0x48, 0x3b, + 0x75, 0x21, 0xa9, 0x04, 0x37, 0xe5, 0x4c, 0x32, 0x5c, 0xf5, 0x99, 0xdf, 0xe3, 0x8c, 0xfa, 0x5d, + 0xb7, 0x37, 0x74, 0x33, 0x91, 0x2b, 0x24, 0xe3, 0x34, 0x84, 0xb4, 0xb3, 0xf9, 0x78, 0xb2, 0xac, + 0x43, 0x12, 0x46, 0x09, 0xa4, 0x9d, 0x7a, 0x7f, 0xe8, 0xfb, 0x26, 0x7a, 0xf3, 0xb1, 0x8e, 0x4c, + 0x3b, 0xf5, 0x28, 0x91, 0xc0, 0x13, 0x1a, 0x7b, 0x9c, 0x9e, 0xca, 0x09, 0xf9, 0x30, 0x23, 0xfb, + 0x20, 0x69, 0x40, 0x25, 0x9d, 0xe0, 0x38, 0xc3, 0x67, 0x30, 0x67, 0x20, 0xa3, 0xb8, 0xde, 0x8d, + 0xfd, 0xba, 0x8c, 0xfa, 0x20, 0x24, 0xed, 0xa7, 0x13, 0x66, 0x3d, 0x64, 0x21, 0xd3, 0xcb, 0xba, + 0x5a, 0x19, 0xb4, 0xf6, 0xa7, 0x8d, 0x56, 0x08, 0xa4, 0x71, 0xe4, 0xd3, 0x17, 0xea, 0x36, 0xf8, + 0x29, 0xc2, 0x6a, 0x6b, 0x8f, 0xa6, 0x69, 0x1c, 0x41, 0xe0, 0x45, 0x49, 0x00, 0x67, 0x8e, 0xb5, + 0x6d, 0xed, 0xd8, 0xa4, 0xac, 0x98, 0x3d, 0x43, 0xb4, 0x14, 0x8e, 0x5d, 0xf4, 0x49, 0x0c, 0x54, + 0xc0, 0x0d, 0xf9, 0xa2, 0x96, 0xdf, 0xd7, 0xd4, 0x9c, 0xfe, 0x1b, 0x64, 0x07, 0x20, 0x7c, 0x27, + 0xb7, 0x6d, 0xed, 0x94, 0x76, 0x6b, 0xee, 0x75, 0xd1, 0x26, 0x77, 0x71, 0x09, 0x4d, 0x42, 0xd8, + 0x07, 0xe1, 0xf3, 0x28, 0x95, 0x8c, 0x13, 0xad, 0xc7, 0x2e, 0xca, 0xeb, 0x64, 0x8e, 0xad, 0x03, + 0x9d, 0x3b, 0x02, 0x0f, 0x15, 0x4f, 0x8c, 0x0c, 0x1f, 0xa3, 0x35, 0xc9, 0x07, 0x89, 0x4f, 0x25, + 0x04, 0x9e, 0xb6, 0xc9, 0xc9, 0xeb, 0xc8, 0x4f, 0xef, 0xdc, 0xf2, 0x54, 0xb6, 0x33, 0xb5, 0xae, + 0x02, 0xb9, 0x27, 0xe7, 0xbe, 0xf1, 0x09, 0x5a, 0x09, 0x7d, 0x4f, 0x76, 0x39, 0x88, 0x2e, 0x8b, + 0x03, 0x67, 0x49, 0x27, 0xdb, 0x9a, 0x49, 0xa6, 0xea, 0xee, 0x76, 0x63, 0xdf, 0x6d, 0x67, 0x75, + 0x6f, 0xac, 0x8d, 0x47, 0xd5, 0xd2, 0x2f, 0xcd, 0x76, 0x16, 0x45, 0x4a, 0xa1, 0x3f, 0xfd, 0xc0, + 0x3f, 0xa0, 0xbc, 0x3a, 0x98, 0x70, 0x96, 0x6f, 0x1d, 0x6c, 0xd2, 0x29, 0x6e, 0xd6, 0x29, 0xee, + 0xd1, 0xcb, 0x66, 0x53, 0x1d, 0x44, 0x10, 0x13, 0x83, 0xbf, 0x46, 0x1b, 0x03, 0x11, 0x25, 0xe1, + 0xb4, 0xee, 0xfa, 0x8e, 0x5e, 0x0f, 0xce, 0x9d, 0xd2, 0xb6, 0xb5, 0x53, 0x20, 0xeb, 0x9a, 0x9e, + 0xd4, 0x5e, 0xdf, 0xe1, 0x57, 0x38, 0xc7, 0x5f, 0xa1, 0xe5, 0x21, 0x70, 0x11, 0xb1, 0xc4, 0x59, + 0xd1, 0xbb, 0x6e, 0xde, 0x51, 0x8e, 0x97, 0x46, 0x41, 0x32, 0xe9, 0xf7, 0xf6, 0xbf, 0x7f, 0x54, + 0xad, 0x03, 0xbb, 0x50, 0x28, 0x17, 0x0f, 0xec, 0x42, 0xb1, 0x8c, 0x0e, 0xec, 0x02, 0x2a, 0x97, + 0x6a, 0x7f, 0x15, 0x50, 0x51, 0xdb, 0xd5, 0x4a, 0x4e, 0x19, 0x3e, 0x32, 0xf7, 0x01, 0xdd, 0x2b, + 0xa5, 0xdd, 0x2f, 0xdc, 0x0f, 0x3c, 0x08, 0x77, 0xb6, 0xed, 0x1a, 0x85, 0x8b, 0x51, 0x75, 0xe1, + 0x72, 0x54, 0xb5, 0xcc, 0x0d, 0x01, 0x6f, 0x21, 0x14, 0x53, 0x21, 0xe7, 0x1a, 0xaa, 0xa8, 0x10, + 0xd3, 0x48, 0x55, 0x54, 0x4a, 0x06, 0x7d, 0x2f, 0x85, 0x24, 0x88, 0x92, 0x50, 0xf7, 0x93, 0x4d, + 0x50, 0x32, 0xe8, 0x3f, 0x37, 0x48, 0x26, 0x08, 0x38, 0x4b, 0x53, 0x08, 0xb4, 0xfb, 0x46, 0xb0, + 0x6f, 0x10, 0x5c, 0x43, 0xab, 0xba, 0xd1, 0x63, 0x16, 0x7a, 0x22, 0x7a, 0x0b, 0xda, 0xd3, 0x1c, + 0x29, 0x29, 0xf0, 0x90, 0x85, 0x2f, 0xa2, 0xb7, 0x80, 0x7f, 0x44, 0x9b, 0x34, 0x4d, 0x39, 0x3b, + 0x8b, 0xfa, 0xaa, 0xbc, 0x29, 0x67, 0x29, 0x13, 0x34, 0xf6, 0x5e, 0x0f, 0x98, 0xa4, 0xda, 0xb8, + 0x1c, 0x71, 0x66, 0x14, 0xcf, 0x27, 0x82, 0x13, 0xc5, 0xe3, 0xcf, 0xd0, 0x1a, 0x57, 0xe5, 0xf1, + 0xfa, 0xf4, 0xcc, 0xeb, 0x9c, 0x4b, 0x10, 0x4e, 0x41, 0x87, 0xac, 0x6a, 0xf8, 0x88, 0x9e, 0x35, + 0x14, 0x88, 0xbf, 0x44, 0x0f, 0xe6, 0x4e, 0xe2, 0x49, 0x3e, 0x10, 0x12, 0x02, 0x07, 0x69, 0x2b, + 0xf1, 0xcc, 0x89, 0xda, 0x86, 0xc1, 0x29, 0xda, 0x48, 0xe0, 0x0d, 0x08, 0xe9, 0xf9, 0x31, 0x13, + 0x10, 0x78, 0xd3, 0xd7, 0xae, 0xfd, 0x2f, 0xed, 0xee, 0x7e, 0xb8, 0xfc, 0x99, 0x73, 0x6e, 0xb3, + 0xfd, 0x73, 0x22, 0xf9, 0x79, 0xc3, 0x56, 0x1e, 0x90, 0x07, 0x26, 0x71, 0x53, 0xe7, 0x9d, 0x36, + 0x33, 0xfe, 0x0d, 0x6d, 0x50, 0x5f, 0x46, 0x43, 0xb8, 0xbd, 0xe3, 0xca, 0xc7, 0x3c, 0x86, 0x49, + 0x72, 0x93, 0xe3, 0x66, 0xf2, 0x6f, 0xd1, 0x86, 0x2e, 0xc9, 0x29, 0x40, 0xe0, 0x71, 0x08, 0x23, + 0x21, 0x39, 0x95, 0x11, 0x4b, 0x84, 0xb3, 0xaa, 0x2b, 0xf6, 0x70, 0x4a, 0x93, 0x59, 0x16, 0x7f, + 0x87, 0x1e, 0xcd, 0x9b, 0xe2, 0x75, 0xd4, 0x34, 0x32, 0x4d, 0x73, 0xcf, 0x84, 0xa6, 0xb3, 0xa6, + 0x34, 0xa8, 0x00, 0xd3, 0x41, 0x3f, 0xa1, 0x27, 0x37, 0x42, 0x39, 0x98, 0x59, 0xf6, 0x7a, 0x00, + 0x03, 0x70, 0xd6, 0xb6, 0x73, 0x3b, 0x39, 0xf2, 0x68, 0x2e, 0x9a, 0x18, 0xc5, 0x89, 0x12, 0xe0, + 0xcf, 0x51, 0x51, 0x42, 0x42, 0x13, 0xe9, 0x45, 0x81, 0x53, 0x56, 0xfd, 0xd5, 0x58, 0x19, 0x8f, + 0xaa, 0x85, 0xb6, 0x06, 0x5b, 0xfb, 0xa4, 0x60, 0xe8, 0x56, 0xa0, 0x9b, 0x99, 0xf9, 0x3d, 0x4f, + 0xd2, 0x4e, 0x0c, 0xce, 0xfd, 0x6d, 0x6b, 0xa7, 0x48, 0x8a, 0x0a, 0x69, 0x2b, 0x60, 0xf3, 0x3f, + 0x0b, 0x2d, 0x4f, 0x4c, 0xc0, 0xaf, 0xd0, 0x72, 0xc2, 0x02, 0x50, 0x39, 0xd5, 0x43, 0xca, 0x37, + 0xf6, 0xc6, 0xa3, 0xea, 0xd2, 0x31, 0x0b, 0xa0, 0xb5, 0xff, 0xff, 0xa8, 0xfa, 0x2c, 0x8c, 0x64, + 0x77, 0xd0, 0x71, 0x7d, 0xd6, 0xaf, 0x4f, 0xeb, 0x1d, 0x74, 0xae, 0xd7, 0xf5, 0xb4, 0x17, 0xd6, + 0xb3, 0xa7, 0x6c, 0xc2, 0xc8, 0x92, 0xca, 0xd8, 0x0a, 0xf0, 0x31, 0x2a, 0xdf, 0x32, 0x6f, 0xf1, + 0xe3, 0xcd, 0x5b, 0xf3, 0x6f, 0xd8, 0xf6, 0x04, 0xd9, 0xfd, 0x98, 0x46, 0xfa, 0xf5, 0xe5, 0x1a, + 0x85, 0xf1, 0xa8, 0x6a, 0x1f, 0x1d, 0xee, 0xb5, 0x88, 0x46, 0xf1, 0x3a, 0xca, 0x43, 0xca, 0xfc, + 0xae, 0x9e, 0xd9, 0x39, 0x62, 0x3e, 0xcc, 0x30, 0x99, 0x8e, 0x14, 0xbb, 0x9c, 0x37, 0x23, 0xa5, + 0x46, 0x50, 0xf9, 0x90, 0x4a, 0xbf, 0x7b, 0x44, 0x13, 0x1a, 0x02, 0xd7, 0x23, 0x65, 0x0b, 0x21, + 0x0e, 0x34, 0xf0, 0x7c, 0x36, 0x48, 0xa4, 0x2e, 0x47, 0x8e, 0x14, 0x15, 0xd2, 0x54, 0x80, 0x7a, + 0xe2, 0x6f, 0x78, 0x24, 0x61, 0xc2, 0x2f, 0x6a, 0x1e, 0x69, 0x48, 0x0b, 0x1a, 0x4f, 0x2f, 0xfe, + 0xa9, 0x2c, 0x5c, 0x8c, 0x2b, 0xd6, 0xe5, 0xb8, 0x62, 0xbd, 0x1b, 0x57, 0xac, 0xbf, 0xc7, 0x15, + 0xeb, 0xf7, 0xab, 0xca, 0xc2, 0xe5, 0x55, 0x65, 0xe1, 0xdd, 0x55, 0x65, 0xe1, 0x15, 0xba, 0xfe, + 0x9b, 0x77, 0x96, 0xf4, 0x1f, 0xf1, 0xd9, 0xfb, 0x00, 0x00, 0x00, 0xff, 0xff, 0xd2, 0x8a, 0xae, + 0x49, 0xee, 0x07, 0x00, 0x00, } diff --git a/pkg/kv/kvserver/kvserverpb/state.proto b/pkg/kv/kvserver/kvserverpb/state.proto index bfd0dd45d65b..f8288300aeac 100644 --- a/pkg/kv/kvserver/kvserverpb/state.proto +++ b/pkg/kv/kvserver/kvserverpb/state.proto @@ -68,6 +68,37 @@ message ReplicaState { // is idempotent by Replica state machines, meaning that it is ok for multiple // Raft commands to set it to true. bool using_applied_state_key = 11; + // Version tells us which migrations can be assumed to have run against this + // particular replica. When we introduce backwards incompatible changes to the + // replica state (for example using the unreplicated truncated state instead + // of the replicated variant), this field tells us if we should expect a + // state representation prior to, or after the change (in our example it + // tells us whether or not we can assume an unreplicated truncated state). All + // migrations associated with versions less than or equal to this one, can be + // assumed to have run. + // + // We should note that the progression of cluster versions in the cluster does + // not happen in lock step with replica versions for all replicas in the + // system. Well, not for all cluster versions. Replica versions are one the + // primitives used to coordinate below-raft migrations, and when cluster + // versions for those are being rolled out cluster-wide, the expectation is + // that all replica versions are bumped along side it. For everything else, + // the coupling is neither necessary nor enforced. When a new replica is being + // initialized, it's annotated with a replica version. This happens in one of + // two places: + // - during cluster creation time, in which case we source the binary version + // (i.e. the version the cluster is bootstrapped with) + // - replicas created through snapshots or splits, where we inherit the + // version from the incoming snapshot or the LHS of the split respectively. + // + // While a below-raft migration is underway, replicas from different ranges + // may have differing replica versions. This is fine, we rely on higher level + // orchestration primitives[1][2] to shepherd all replicas in the system to + // the right version. + // + // [1]: migration.Manager + // [2]: PurgeOutdatedReplicas + roachpb.Version version = 12; reserved 8, 9, 10; } diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 1f6b68237a53..1367bf2221b2 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -798,6 +798,13 @@ func (r *Replica) GetGCThreshold() hlc.Timestamp { return *r.mu.state.GCThreshold } +// Version returns the replica version. +func (r *Replica) Version() roachpb.Version { + r.mu.RLock() + defer r.mu.RUnlock() + return *r.mu.state.Version +} + // getImpliedGCThresholdRLocked returns the gc threshold of the replica which // should be used to determine the validity of commands. The returned timestamp // may be newer than the replica's true GC threshold if strict enforcement diff --git a/pkg/kv/kvserver/replica_application_result.go b/pkg/kv/kvserver/replica_application_result.go index dea3a95ce7e1..598dc74f5c16 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -293,6 +293,15 @@ func (r *Replica) handleGCThresholdResult(ctx context.Context, thresh *hlc.Times r.mu.Unlock() } +func (r *Replica) handleVersionResult(ctx context.Context, version *roachpb.Version) { + if (*version == roachpb.Version{}) { + log.Fatal(ctx, "not expecting empty replica version downstream of raft") + } + r.mu.Lock() + r.mu.state.Version = version + r.mu.Unlock() +} + func (r *Replica) handleUsingAppliedStateKeyResult(ctx context.Context) { r.mu.Lock() r.mu.state.UsingAppliedStateKey = true diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index ad02664c0565..c3bdbf3fc174 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -15,6 +15,7 @@ import ( "fmt" "time" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -699,8 +700,19 @@ func (b *replicaAppBatch) runPreApplyTriggersAfterStagingWriteBatch( } if res.State != nil && res.State.TruncatedState != nil { + activeVersion := b.r.ClusterSettings().Version.ActiveVersion(ctx).Version + migrationVersion := clusterversion.ByKey(clusterversion.TruncatedAndRangeAppliedStateMigration) + // NB: We're being deliberate here in using the less-than operator (as + // opposed to LessEq). TruncatedAndRangeAppliedStateMigration indicates + // that the migration to move to the unreplicated truncated + // state is currently underway. It's only when the active cluster + // version has moved past it that we can assume that the migration has + // completed. + assertNoLegacy := migrationVersion.Less(activeVersion) + if apply, err := handleTruncatedStateBelowRaft( ctx, b.state.TruncatedState, res.State.TruncatedState, b.r.raftMu.stateLoader, b.batch, + assertNoLegacy, ); err != nil { return wrapWithNonDeterministicFailure(err, "unable to handle truncated state") } else if !apply { @@ -1112,6 +1124,10 @@ func (sm *replicaStateMachine) handleNonTrivialReplicatedEvalResult( rResult.State.GCThreshold = nil } + if newVersion := rResult.State.Version; newVersion != nil { + sm.r.handleVersionResult(ctx, newVersion) + rResult.State.Version = nil + } if (*rResult.State == kvserverpb.ReplicaState{}) { rResult.State = nil } diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index 0fbd20a1693d..3b4b4d01ce4b 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -773,12 +773,23 @@ func (r *Replica) evaluateProposal( usingAppliedStateKey := r.mu.state.UsingAppliedStateKey r.mu.RUnlock() if !usingAppliedStateKey { + // The range applied state was originally introduced in v2.1, and in + // v21.1 we guarantee that it's used for all ranges, which we assert + // on below. If we're not running 21.1 yet, migrate over as we've + // done since the introduction of the applied state key. + activeVersion := r.ClusterSettings().Version.ActiveVersion(ctx).Version + migrationVersion := clusterversion.ByKey(clusterversion.TruncatedAndRangeAppliedStateMigration) + if migrationVersion.Less(activeVersion) { + log.Fatalf(ctx, "not using applied state key in v21.1") + } // The range applied state was introduced in v2.1. It's possible to // still find ranges that haven't activated it. If so, activate it. // We can remove this code if we introduce a boot-time check that // fails the startup process when any legacy replicas are found. The // operator can then run the old binary for a while to upgrade the // stragglers. + // + // TODO(irfansharif): Is this still applicable? if res.Replicated.State == nil { res.Replicated.State = &kvserverpb.ReplicaState{} } diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 99cae0ee4884..d7255a74ca42 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -1672,6 +1672,7 @@ func handleTruncatedStateBelowRaft( oldTruncatedState, newTruncatedState *roachpb.RaftTruncatedState, loader stateloader.StateLoader, readWriter storage.ReadWriter, + assertNoLegacy bool, ) (_apply bool, _ error) { // If this is a log truncation, load the resulting unreplicated or legacy // replicated truncated state (in that order). If the migration is happening @@ -1686,6 +1687,10 @@ func handleTruncatedStateBelowRaft( return false, errors.Wrap(err, "loading truncated state") } + if assertNoLegacy && truncStateIsLegacy { + log.Fatalf(ctx, "found legacy truncated state which should no longer exist") + } + // Truncate the Raft log from the entry after the previous // truncation index to the new truncation index. This is performed // atomically with the raft command application so that the diff --git a/pkg/kv/kvserver/replica_raft_truncation_test.go b/pkg/kv/kvserver/replica_raft_truncation_test.go index 965cb95721c6..3f77daf63e8d 100644 --- a/pkg/kv/kvserver/replica_raft_truncation_test.go +++ b/pkg/kv/kvserver/replica_raft_truncation_test.go @@ -87,7 +87,7 @@ func TestHandleTruncatedStateBelowRaft(t *testing.T) { Term: term, } - apply, err := handleTruncatedStateBelowRaft(ctx, &prevTruncatedState, newTruncatedState, loader, eng) + apply, err := handleTruncatedStateBelowRaft(ctx, &prevTruncatedState, newTruncatedState, loader, eng, false) if err != nil { return err.Error() } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 4f6d43585c7f..f8f5c035995a 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -271,6 +271,7 @@ func (tc *testContext) StartWithStoreConfigAndVersion( nil, /* initialValues */ bootstrapVersion, 1 /* numStores */, nil /* splits */, cfg.Clock.PhysicalNow(), + cfg.TestingKnobs, ); err != nil { t.Fatal(err) } @@ -286,7 +287,9 @@ func (tc *testContext) StartWithStoreConfigAndVersion( if realRange { if tc.bootstrapMode == bootstrapRangeOnly { testDesc := testRangeDescriptor() - if err := stateloader.WriteInitialRangeState(ctx, tc.store.Engine(), *testDesc); err != nil { + if err := stateloader.WriteInitialRangeState( + ctx, tc.store.Engine(), *testDesc, roachpb.Version{}, + ); err != nil { t.Fatal(err) } repl, err := newReplica(ctx, testDesc, tc.store, 1) diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index 1c8edfceec9e..049060f8685a 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -216,6 +216,59 @@ 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. + // + // 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 + // (which include learner replicas currently in the process of + // receiving snapshots). 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. We also have a + // separate mechanism to ensure replicas with older versions are + // purged from the system[1]. This is driven by a higher-level + // orchestration layer[2], these are the replicas that we don't + // have a handle on here as they're eligible for GC (but may + // still hit replica evaluation code paths with pre-migrated + // state, unless explicitly purged). + // + // It's possible that between the proposal returning and the + // call to r.Desc() below, the descriptor has already changed. + // But the only thing that matters is that r.Desc() is at least + // as up to date as the descriptor the command applied on + // previously. If a replica got removed - fine, + // waitForApplication will fail; we will have to cope with that. + // If one got added - it was likely already a learner when we + // migrated (in which case waitForApplication will know about + // it). If that's not the case, we'll note that the Migrate + // command also declares a read latch on the range descriptor. + // The replication change will have thus serialized after the + // migration, and so the snapshot will also include the + // post-migration state. + // + // TODO(irfansharif): In a cluster that is constantly changing + // its replica sets, it's possible to get into a situation + // where a Migrate command never manages to complete - all it + // takes is a single range in each attempt to throw things off. + // Perhaps an error in waitForApplication should lead to a retry + // of just the one RPC instead of propagating an error for the + // entire migrate invocation. + // + // [1]: See PurgeOutdatedReplicas from the Migration service. + // [2]: pkg/migration + 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/kv/kvserver/stateloader/initial.go b/pkg/kv/kvserver/stateloader/initial.go index 9fd76fa8568e..65177fea2c69 100644 --- a/pkg/kv/kvserver/stateloader/initial.go +++ b/pkg/kv/kvserver/stateloader/initial.go @@ -46,6 +46,7 @@ func WriteInitialReplicaState( lease roachpb.Lease, gcThreshold hlc.Timestamp, truncStateType TruncatedStateType, + replicaVersion roachpb.Version, ) (enginepb.MVCCStats, error) { rsl := Make(desc.RangeID) var s kvserverpb.ReplicaState @@ -60,7 +61,12 @@ func WriteInitialReplicaState( s.Stats = &ms s.Lease = &lease s.GCThreshold = &gcThreshold - s.UsingAppliedStateKey = true + if (replicaVersion != roachpb.Version{}) { + s.Version = &replicaVersion + } + if truncStateType != TruncatedStateLegacyReplicatedAndNoAppliedKey { + s.UsingAppliedStateKey = true + } if existingLease, err := rsl.LoadLease(ctx, readWriter); err != nil { return enginepb.MVCCStats{}, errors.Wrap(err, "error reading lease") @@ -71,7 +77,13 @@ func WriteInitialReplicaState( if existingGCThreshold, err := rsl.LoadGCThreshold(ctx, readWriter); err != nil { return enginepb.MVCCStats{}, errors.Wrap(err, "error reading GCThreshold") } else if !existingGCThreshold.IsEmpty() { - log.Fatalf(ctx, "expected trivial GChreshold, but found %+v", existingGCThreshold) + log.Fatalf(ctx, "expected trivial GCthreshold, but found %+v", existingGCThreshold) + } + + if existingVersion, err := rsl.LoadVersion(ctx, readWriter); err != nil { + return enginepb.MVCCStats{}, errors.Wrap(err, "error reading Version") + } else if (existingVersion != roachpb.Version{}) { + log.Fatalf(ctx, "expected trivial version, but found %+v", existingVersion) } newMS, err := rsl.Save(ctx, readWriter, s, truncStateType) @@ -85,15 +97,36 @@ func WriteInitialReplicaState( // WriteInitialRangeState writes the initial range state. It's called during // bootstrap. func WriteInitialRangeState( - ctx context.Context, readWriter storage.ReadWriter, desc roachpb.RangeDescriptor, + ctx context.Context, + readWriter storage.ReadWriter, + desc roachpb.RangeDescriptor, + replicaVersion roachpb.Version, +) error { + const initialTruncStateType = TruncatedStateUnreplicated + return WriteInitialRangeStateWithTruncatedState(ctx, readWriter, desc, replicaVersion, initialTruncStateType) +} + +// WriteInitialRangeStateWithTruncatedState is the same as +// WriteInitialRangeState, but allows the caller to override the truncated state +// type. +// +// TODO(irfansharif): This can be removed in the v21.2 cycle after we no longer +// need to test the truncated state migration. +func WriteInitialRangeStateWithTruncatedState( + ctx context.Context, + readWriter storage.ReadWriter, + desc roachpb.RangeDescriptor, + replicaVersion roachpb.Version, + truncState TruncatedStateType, ) error { initialLease := roachpb.Lease{} initialGCThreshold := hlc.Timestamp{} - initialTruncStateType := TruncatedStateUnreplicated initialMS := enginepb.MVCCStats{} + initialTruncStateType := truncState if _, err := WriteInitialReplicaState( - ctx, readWriter, initialMS, desc, initialLease, initialGCThreshold, initialTruncStateType, + ctx, readWriter, initialMS, desc, initialLease, initialGCThreshold, + initialTruncStateType, replicaVersion, ); err != nil { return err } diff --git a/pkg/kv/kvserver/stateloader/stateloader.go b/pkg/kv/kvserver/stateloader/stateloader.go index 3230b064314b..9a1a4d5be558 100644 --- a/pkg/kv/kvserver/stateloader/stateloader.go +++ b/pkg/kv/kvserver/stateloader/stateloader.go @@ -103,6 +103,14 @@ func (rsl StateLoader) Load( } s.TruncatedState = &truncState + version, err := rsl.LoadVersion(ctx, reader) + if err != nil { + return kvserverpb.ReplicaState{}, err + } + if (version != roachpb.Version{}) { + s.Version = &version + } + return s, nil } @@ -113,6 +121,9 @@ type TruncatedStateType int const ( // TruncatedStateLegacyReplicated means use the legacy (replicated) key. TruncatedStateLegacyReplicated TruncatedStateType = iota + // TruncatedStateLegacyReplicatedAndNoAppliedKey means use the legacy key + // and also don't use the RangeAppliedKey. This is for testing use only. + TruncatedStateLegacyReplicatedAndNoAppliedKey // TruncatedStateUnreplicated means use the new (unreplicated) key. TruncatedStateUnreplicated ) @@ -141,7 +152,7 @@ func (rsl StateLoader) Save( if err := rsl.SetGCThreshold(ctx, readWriter, ms, state.GCThreshold); err != nil { return enginepb.MVCCStats{}, err } - if truncStateType == TruncatedStateLegacyReplicated { + if truncStateType != TruncatedStateUnreplicated { if err := rsl.SetLegacyRaftTruncatedState(ctx, readWriter, ms, state.TruncatedState); err != nil { return enginepb.MVCCStats{}, err } @@ -150,6 +161,11 @@ func (rsl StateLoader) Save( return enginepb.MVCCStats{}, err } } + if state.Version != nil { + if err := rsl.SetVersion(ctx, readWriter, ms, state.Version); err != nil { + return enginepb.MVCCStats{}, err + } + } if state.UsingAppliedStateKey { rai, lai := state.RaftAppliedIndex, state.LeaseAppliedIndex if err := rsl.SetRangeAppliedState(ctx, readWriter, rai, lai, ms); err != nil { @@ -505,6 +521,27 @@ func (rsl StateLoader) SetGCThreshold( rsl.RangeLastGCKey(), hlc.Timestamp{}, nil, threshold) } +// LoadVersion loads the replica version. +func (rsl StateLoader) LoadVersion( + ctx context.Context, reader storage.Reader, +) (roachpb.Version, error) { + var version roachpb.Version + _, err := storage.MVCCGetProto(ctx, reader, rsl.RangeVersionKey(), + hlc.Timestamp{}, &version, storage.MVCCGetOptions{}) + return version, err +} + +// SetVersion sets the replica version. +func (rsl StateLoader) SetVersion( + ctx context.Context, + readWriter storage.ReadWriter, + ms *enginepb.MVCCStats, + version *roachpb.Version, +) error { + return storage.MVCCPutProto(ctx, readWriter, ms, + rsl.RangeVersionKey(), hlc.Timestamp{}, nil, version) +} + // The rest is not technically part of ReplicaState. // LoadLastIndex loads the last index. diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 9845503c07bc..ab1bfabf3680 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -54,6 +54,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/cloud" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/contextutil" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/iterutil" @@ -2797,6 +2798,58 @@ func (s *Store) ManuallyEnqueue( return collect(), processErr, nil } +// PurgeOutdatedReplicas purges all replicas with a version less than the one +// specified. This entails clearing out replicas in the replica GC queue that +// fit the bill. +func (s *Store) PurgeOutdatedReplicas(ctx context.Context, version roachpb.Version) error { + if interceptor := s.TestingKnobs().PurgeOutdatedReplicasInterceptor; interceptor != nil { + interceptor() + } + + // Let's set a reasonable bound on the number of replicas being processed in + // parallel. + qp := quotapool.NewIntPool("purge-outdated-replicas", 50) + g := ctxgroup.WithContext(ctx) + s.VisitReplicas(func(repl *Replica) (wantMore bool) { + if !repl.Version().Less(version) { + // Nothing to do here. + return true + } + + alloc, err := qp.Acquire(ctx, 1) + if err != nil { + g.GoCtx(func(ctx context.Context) error { + return err + }) + return false + } + + g.GoCtx(func(ctx context.Context) error { + defer alloc.Release() + + processed, err := s.replicaGCQueue.process(ctx, repl, nil) + if err != nil { + return errors.Wrapf(err, "on %s", repl.Desc()) + } + if !processed { + // We're either still part of the raft group, in which same + // something has gone horribly wrong, or more likely (though + // still very unlikely in practice): this range has been merged + // away, and this store has the replica of the subsuming range + // where we're unable to determine if it has applied the merge + // trigger. See replicaGCQueue.process for more details. Either + // way, we error out. + return errors.Newf("unable to gc %s", repl.Desc()) + } + return nil + }) + + return true + }) + + return g.Wait() +} + // WriteClusterVersion writes the given cluster version to the store-local // cluster version key. We only accept a raw engine to ensure we're persisting // the write durably. diff --git a/pkg/kv/kvserver/store_create_replica.go b/pkg/kv/kvserver/store_create_replica.go index 2044d4d31b51..f50b97c741d7 100644 --- a/pkg/kv/kvserver/store_create_replica.go +++ b/pkg/kv/kvserver/store_create_replica.go @@ -180,6 +180,7 @@ func (s *Store) tryGetOrCreateReplica( // been set, not every code path which inspects the descriptor checks the // destroy status. repl.mu.state.Desc = uninitializedDesc + // Add the range to range map, but not replicasByKey since the range's start // key is unknown. The range will be added to replicasByKey later when a // snapshot is applied. After unlocking Store.mu above, another goroutine diff --git a/pkg/kv/kvserver/store_init.go b/pkg/kv/kvserver/store_init.go index 00679438e04b..1580744e3a58 100644 --- a/pkg/kv/kvserver/store_init.go +++ b/pkg/kv/kvserver/store_init.go @@ -84,6 +84,7 @@ func WriteInitialClusterData( numStores int, splits []roachpb.RKey, nowNanos int64, + knobs StoreTestingKnobs, ) error { // Bootstrap version information. We'll add the "bootstrap version" to the // list of initialValues, so that we don't have to handle it specially @@ -233,8 +234,16 @@ func WriteInitialClusterData( } } - if err := stateloader.WriteInitialRangeState(ctx, batch, *desc); err != nil { - return err + if tt := knobs.TruncatedStateTypeOverride; tt != nil { + if err := stateloader.WriteInitialRangeStateWithTruncatedState( + ctx, batch, *desc, bootstrapVersion, *tt, + ); err != nil { + return err + } + } else { + if err := stateloader.WriteInitialRangeState(ctx, batch, *desc, bootstrapVersion); err != nil { + return err + } } computedStats, err := rditer.ComputeStatsForRange(desc, batch, now.WallTime) if err != nil { diff --git a/pkg/kv/kvserver/store_raft.go b/pkg/kv/kvserver/store_raft.go index bf4b4e4a3005..5a54ab903003 100644 --- a/pkg/kv/kvserver/store_raft.go +++ b/pkg/kv/kvserver/store_raft.go @@ -296,9 +296,9 @@ func (s *Store) processRaftSnapshotRequest( defer s.mu.Unlock() placeholder, err := s.canApplySnapshotLocked(ctx, snapHeader) if err != nil { - // If the storage cannot accept the snapshot, return an - // error before passing it to RawNode.Step, since our - // error handling options past that point are limited. + // If we cannot accept the snapshot, return an error before + // passing it to RawNode.Step, since our error handling options + // past that point are limited. log.Infof(ctx, "cannot apply snapshot: %s", err) return err } diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index b89302f802f1..5860b36394f3 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -16,6 +16,7 @@ import ( "io" "time" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/raftentry" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" @@ -926,6 +927,11 @@ func SendEmptySnapshot( ); err != nil { return err } + + var replicaVersion roachpb.Version + if st.Version.IsActive(ctx, clusterversion.ReplicaVersions) { + replicaVersion = st.Version.ActiveVersionOrEmpty(ctx).Version + } ms, err := stateloader.WriteInitialReplicaState( ctx, eng, @@ -934,6 +940,7 @@ func SendEmptySnapshot( roachpb.Lease{}, hlc.Timestamp{}, // gcThreshold stateloader.TruncatedStateUnreplicated, + replicaVersion, ) if err != nil { return err diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index dfaaa735c021..f1cc4c44f6b8 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -256,7 +256,7 @@ func createTestStoreWithoutStart( if err := WriteInitialClusterData( context.Background(), eng, kvs, /* initialValues */ clusterversion.TestingBinaryVersion, - 1 /* numStores */, splits, cfg.Clock.PhysicalNow(), + 1 /* numStores */, splits, cfg.Clock.PhysicalNow(), cfg.TestingKnobs, ); err != nil { t.Fatal(err) } @@ -472,7 +472,7 @@ func TestStoreInitAndBootstrap(t *testing.T) { if err := WriteInitialClusterData( ctx, eng, kvs /* initialValues */, clusterversion.TestingBinaryVersion, - 1 /* numStores */, splits, cfg.Clock.PhysicalNow(), + 1 /* numStores */, splits, cfg.Clock.PhysicalNow(), cfg.TestingKnobs, ); err != nil { t.Errorf("failure to create first range: %+v", err) } @@ -1424,7 +1424,7 @@ func splitTestRange(store *Store, key, splitKey roachpb.RKey, t *testing.T) *Rep rangeID, splitKey, repl.Desc().EndKey, repl.Desc().Replicas()) // Minimal amount of work to keep this deprecated machinery working: Write // some required Raft keys. - err = stateloader.WriteInitialRangeState(ctx, store.engine, *rhsDesc) + err = stateloader.WriteInitialRangeState(ctx, store.engine, *rhsDesc, roachpb.Version{}) require.NoError(t, err) newRng, err := newReplica(ctx, rhsDesc, store, repl.ReplicaID()) require.NoError(t, err) @@ -2929,7 +2929,9 @@ func TestStoreRemovePlaceholderOnRaftIgnored(t *testing.T) { } uninitDesc := roachpb.RangeDescriptor{RangeID: repl1.Desc().RangeID} - if err := stateloader.WriteInitialRangeState(ctx, s.Engine(), uninitDesc); err != nil { + if err := stateloader.WriteInitialRangeState( + ctx, s.Engine(), uninitDesc, roachpb.Version{}, + ); err != nil { t.Fatal(err) } uninitRepl1, err := newReplica(ctx, &uninitDesc, s, 2) diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index a7e1bb014537..f909ff699908 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/tenantrate" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -263,6 +264,12 @@ type StoreTestingKnobs struct { // even when the replicate queue is enabled. This often results in flaky // tests, so by default, it is prevented. AllowUnsynchronizedReplicationChanges bool + // PurgeOutdatedReplicasInterceptor intercepts attempts to purge outdated + // replicas in the store. + PurgeOutdatedReplicasInterceptor func() + // If set, use the given truncated state type when bootstrapping ranges. + // This is used for testing the truncated state migration. + TruncatedStateTypeOverride *stateloader.TruncatedStateType } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. diff --git a/pkg/migration/BUILD.bazel b/pkg/migration/BUILD.bazel index 95c130135703..22c1fca8b88b 100644 --- a/pkg/migration/BUILD.bazel +++ b/pkg/migration/BUILD.bazel @@ -37,25 +37,34 @@ go_test( "client_test.go", "helper_test.go", "main_test.go", + "migrations_test.go", "util_test.go", ], embed = [":migration"], deps = [ + "//pkg/base", "//pkg/clusterversion", "//pkg/kv", "//pkg/kv/kvserver", + "//pkg/kv/kvserver/batcheval", + "//pkg/kv/kvserver/liveness", "//pkg/kv/kvserver/liveness/livenesspb", + "//pkg/kv/kvserver/stateloader", "//pkg/roachpb", "//pkg/security", "//pkg/security/securitytest", "//pkg/server", "//pkg/server/serverpb", + "//pkg/settings/cluster", "//pkg/sql/tests", "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/testcluster", "//pkg/util/leaktest", + "//pkg/util/log", "//pkg/util/syncutil", + "@com_github_cockroachdb_errors//:errors", + "@com_github_stretchr_testify//require", "@org_golang_google_grpc//:grpc", ], ) diff --git a/pkg/migration/helper.go b/pkg/migration/helper.go index 510142b7a162..f877417d22c1 100644 --- a/pkg/migration/helper.go +++ b/pkg/migration/helper.go @@ -70,42 +70,14 @@ 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) -// 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 -// -// [*]: 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 -// 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. -// -// 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 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( +// ForEveryNode is a short hand to execute the given closure (named by the +// informational parameter op) against every node in the cluster at a given +// point in time. Given it's possible for nodes to join or leave the cluster +// during (we don't make any guarantees for the ordering of cluster membership +// events), we only expect this to be used in conjunction with +// UntilClusterStable (see the comment there for how these two primitives can be +// put together). +func (h *Helper) ForEveryNode( ctx context.Context, op string, fn func(context.Context, serverpb.MigrationClient) error, ) error { ns, err := h.c.nodes(ctx) @@ -115,29 +87,79 @@ func (h *Helper) EveryNode( // We'll want to rate limit outgoing RPCs (limit pulled out of thin air). qp := quotapool.NewIntPool("every-node", 25) - for { - log.Infof(ctx, "executing %s on nodes %s", redact.Safe(op), ns) + log.Infof(ctx, "executing %s on nodes %s", redact.Safe(op), ns) + grp := ctxgroup.WithContext(ctx) - grp := ctxgroup.WithContext(ctx) - for _, node := range ns { - id := node.id // copy out of the loop variable - alloc, err := qp.Acquire(ctx, 1) + for _, node := range ns { + id := node.id // copy out of the loop variable + alloc, err := qp.Acquire(ctx, 1) + if err != nil { + return err + } + + grp.GoCtx(func(ctx context.Context) error { + defer alloc.Release() + + conn, err := h.c.dial(ctx, id) if err != nil { return err } + client := serverpb.NewMigrationClient(conn) + return fn(ctx, client) + }) + } + return grp.Wait() +} - grp.GoCtx(func(ctx context.Context) error { - defer alloc.Release() +// UntilClusterStable invokes the given closure until the cluster membership is +// stable, i.e once the set of nodes in the cluster before and after the closure +// are identical, and no nodes have restarted in the interim, we can return to +// the caller[*]. +// +// The mechanism for doing 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) 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), or a node +// restarting +// (d) If there any discrepancies between the list retrieved in (a) +// and (c), we'll invoke the closure again +// (e) We'll continue to loop around until the node ID list stabilizes +// +// [*]: We can be a bit more precise here. What UntilClusterStable gives us is a +// strict causal happened-before relation between running the given closure and +// the next node that joins the cluster. Put another way: using +// UntilClusterStable callers will have managed to run something 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 an example of how this primitive is used, let's consider our use +// of it to bump the cluster version. We use in conjunction with ForEveryNode, +// where after we return, we can rely on the guarantee that all nodes in the +// cluster will have their cluster versions bumped. This then implies that +// future node additions will observe the latest version (through the join RPC). +// That in turn 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. +// +// Given that it'll always be possible for new nodes to join after an +// UntilClusterStable 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 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) UntilClusterStable(ctx context.Context, fn func() error) error { + ns, err := h.c.nodes(ctx) + if err != nil { + return err + } - conn, err := h.c.dial(ctx, id) - if err != nil { - return err - } - client := serverpb.NewMigrationClient(conn) - return fn(ctx, client) - }) - } - if err := grp.Wait(); err != nil { + for { + if err := fn(); err != nil { return err } @@ -230,6 +252,12 @@ func (h *Helper) DB() *kv.DB { return h.c.db() } +// ClusterVersion exposes the cluster version associated with the ongoing +// migration. +func (h *Helper) ClusterVersion() clusterversion.ClusterVersion { + return h.cv +} + type clusterImpl struct { nl nodeLiveness exec sqlutil.InternalExecutor diff --git a/pkg/migration/helper_test.go b/pkg/migration/helper_test.go index 54d52f51f66d..80ffdf9aa822 100644 --- a/pkg/migration/helper_test.go +++ b/pkg/migration/helper_test.go @@ -26,7 +26,7 @@ import ( "google.golang.org/grpc" ) -func TestHelperEveryNode(t *testing.T) { +func TestHelperEveryNodeUntilClusterStable(t *testing.T) { defer leaktest.AfterTest(t) cv := clusterversion.ClusterVersion{} @@ -40,16 +40,18 @@ func TestHelperEveryNode(t *testing.T) { tc := TestingNewCluster(numNodes) 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 + err := h.UntilClusterStable(ctx, func() error { + return h.ForEveryNode(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) @@ -66,16 +68,18 @@ func TestHelperEveryNode(t *testing.T) { tc := TestingNewCluster(numNodes) 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 + err := h.UntilClusterStable(ctx, func() error { + return h.ForEveryNode(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) @@ -94,22 +98,26 @@ func TestHelperEveryNode(t *testing.T) { expRe := fmt.Sprintf("n%d required, but unavailable", downedNode) h := newHelper(tc, cv) opCount := 0 - if err := h.EveryNode(ctx, "dummy-op", func(context.Context, serverpb.MigrationClient) error { - mu.Lock() - defer mu.Unlock() - - opCount++ - if opCount == 1 { - tc.downNode(downedNode) - } - return nil + if err := h.UntilClusterStable(ctx, func() error { + return h.ForEveryNode(ctx, "dummy-op", func(context.Context, serverpb.MigrationClient) error { + mu.Lock() + defer mu.Unlock() + + opCount++ + if opCount == 1 { + tc.downNode(downedNode) + } + return nil + }) }); !testutils.IsError(err, expRe) { t.Fatalf("expected error %q, got %q", expRe, err) } tc.restartNode(downedNode) - if err := h.EveryNode(ctx, "dummy-op", func(context.Context, serverpb.MigrationClient) error { - return nil + if err := h.UntilClusterStable(ctx, func() error { + return h.ForEveryNode(ctx, "dummy-op", func(context.Context, serverpb.MigrationClient) error { + return nil + }) }); err != nil { t.Fatal(err) } diff --git a/pkg/migration/manager.go b/pkg/migration/manager.go index df2e77efe5b6..adf989710c98 100644 --- a/pkg/migration/manager.go +++ b/pkg/migration/manager.go @@ -102,17 +102,61 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe cluster := newCluster(m.nl, m.dialer, m.executor, 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 - // return. The migration associated with the specific version can assume - // that every node in the cluster has the corresponding version - // activated. + // First run the actual migration (if any). The cluster version bump + // will be rolled out afterwards. This lets us provide the invariant + // that if a version=V is active, all data is guaranteed to have + // migrated. + if migration, ok := registry[clusterVersion]; ok { + if err := migration.Run(ctx, h); err != nil { + return err + } + } + + // Next we'll push out the version gate to every node in the cluster. + // Each node will persist the version, bump the local version gates, and + // then return. The migration associated with the specific version is + // executed before every node in the cluster has the corresponding + // version activated. Migrations that depend on a certain version + // already being activated will need to registered using a cluster + // version greater than it. + // + // For each intermediate version, we'll need to first bump the fence + // version before bumping the "real" one. Doing so allows us to provide + // the invariant that whenever a cluster version is active, all nodes in + // the cluster (including ones added concurrently during version + // upgrades) are running binaries that know about the version. + + // Below-raft migrations mutate replica state, making use of the + // Migrate(version=V) primitive which they issue against the entire + // keyspace. These migrations typically want to rely on the invariant + // that there are no extant replicas in the system that haven't seen the + // specific Migrate command. // - // We'll need to first bump the fence version for each intermediate - // cluster version, before bumping the "real" one. Doing so allows us to - // provide the invariant that whenever a cluster version is active, all - // nodes in the cluster (including ones added concurrently during - // version upgrades) are running binaries that know about the version. + // This is partly achieved through the implementation of the Migrate + // command itself, which waits until it's applied on all followers[2] + // before returning. This also addresses the concern of extant snapshots + // with pre-migrated state possibly instantiating older version + // replicas. The intended learner replicas are listed as part of the + // range descriptor, and is also waited on for during command + // application. As for stale snapshots, if they specify a replicaID + // that's no longer part of the raft group, they're discarded by the + // recipient. Snapshots are also discarded unless they move the LAI + // forward. + // + // That still leaves rooms for replicas in the replica GC queue to evade + // detection. To address this, below-raft migrations typically take a + // two-phrase approach (the TruncatedAndRangeAppliedStateMigration being + // one example of this), where after having migrated the entire keyspace + // to version V, and after having prevented subsequent snapshots + // originating from replicas with versions < V, the migration sets out + // to purge outdated replicas in the system[3]. Specifically it + // processes all replicas in the GC queue with a version < V (which are + // not accessible during the application of the Migrate command). + // + // [1]: See ReplicaState.Version. + // [2]: See Replica.executeWriteBatch, specifically how proposals with the + // Migrate request are handled downstream of raft. + // [3]: See PurgeOutdatedReplicas from the Migration service. { // The migrations infrastructure makes use of internal fence @@ -120,12 +164,15 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe // instructive to walk through how we expect a version migration // from v21.1 to v21.2 to take place, and how we behave in the // presence of new v21.1 or v21.2 nodes being added to the cluster. + // // - All nodes are running v21.1 // - All nodes are rolled into v21.2 binaries, but with active // cluster version still as v21.1 // - The first version bump will be into v21.2-1(fence), see the // migration manager above for where that happens + // // Then concurrently: + // // - A new node is added to the cluster, but running binary v21.1 // - We try bumping the cluster gates to v21.2-1(fence) // @@ -140,18 +187,19 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe // by the join RPC). // // All of which is to say that once we've seen the node list - // stabilize (as EveryNode enforces), any new nodes that can join - // the cluster will run a release that support the fence version, - // and by design also supports the actual version (which is the - // direct successor of the fence). + // stabilize (as UntilClusterStable enforces), any new nodes that + // can join the cluster will run a release that support the fence + // version, and by design also supports the actual version (which is + // the direct successor of the fence). fenceVersion := fenceVersionFor(ctx, clusterVersion) req := &serverpb.BumpClusterVersionRequest{ClusterVersion: &fenceVersion} op := fmt.Sprintf("bump-cluster-version=%s", req.ClusterVersion.PrettyPrint()) - err := h.EveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { - _, err := client.BumpClusterVersion(ctx, req) - return err - }) - if err != nil { + if err := h.UntilClusterStable(ctx, func() error { + return h.ForEveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { + _, err := client.BumpClusterVersion(ctx, req) + return err + }) + }); err != nil { return err } } @@ -160,11 +208,12 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe // cluster version bump, cluster-wide. req := &serverpb.ValidateTargetClusterVersionRequest{ClusterVersion: &clusterVersion} op := fmt.Sprintf("validate-cluster-version=%s", req.ClusterVersion.PrettyPrint()) - err := h.EveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { - _, err := client.ValidateTargetClusterVersion(ctx, req) - return err - }) - if err != nil { + if err := h.UntilClusterStable(ctx, func() error { + return h.ForEveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { + _, err := client.ValidateTargetClusterVersion(ctx, req) + return err + }) + }); err != nil { return err } } @@ -172,27 +221,15 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe // Finally, bump the real version cluster-wide. req := &serverpb.BumpClusterVersionRequest{ClusterVersion: &clusterVersion} op := fmt.Sprintf("bump-cluster-version=%s", req.ClusterVersion.PrettyPrint()) - if err := h.EveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { - _, err := client.BumpClusterVersion(ctx, req) - return err + if err := h.UntilClusterStable(ctx, func() error { + return h.ForEveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { + _, err := client.BumpClusterVersion(ctx, req) + return err + }) }); err != nil { return err } } - - // 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. - - // 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 9382ef0abe79..25ea7af8c3a3 100644 --- a/pkg/migration/migrations.go +++ b/pkg/migration/migrations.go @@ -11,9 +11,16 @@ package migration import ( + "bytes" "context" + "fmt" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/logtags" ) // registry defines the global mapping between a cluster version and the @@ -22,9 +29,10 @@ import ( var registry = make(map[clusterversion.ClusterVersion]Migration) func init() { - // TODO(irfansharif): We'll want to register individual migrations with - // specific internal cluster versions here. - _ = register // register(clusterversion.WhateverMigration, WhateverMigration, "whatever migration") + register(clusterversion.TruncatedAndRangeAppliedStateMigration, truncatedStateMigration, + "use unreplicated TruncatedState and RangeAppliedState for all ranges") + register(clusterversion.PostTruncatedAndRangeAppliedStateMigration, postTruncatedStateMigration, + "purge all replicas using the replicated TruncatedState") } // Migration defines a program to be executed once every node in the cluster is @@ -52,7 +60,7 @@ func init() { // // [1]: `(*Manager).Migrate` // [2]: pkg/clusterversion/cockroach_versions.go -// [3]: TruncatedStateMigration +// [3]: truncatedStateMigration // [4]: pkg/kv/kvserver/batch_eval/cmd_migrate.go // // TODO(irfansharif): [3] and [4] are currently referring to what was prototyped @@ -70,9 +78,106 @@ type migrationFn func(context.Context, *Helper) error // // TODO(irfansharif): Introduce a `system.migrations` table, and populate it here. func (m *Migration) Run(ctx context.Context, h *Helper) (err error) { + ctx = logtags.AddTag(ctx, "migration", h.ClusterVersion()) + if err := m.fn(ctx, h); err != nil { return err } return nil } + +// defaultPageSize controls how many range descriptors are paged in by default +// when iterating through all ranges in a cluster during any given migration. We +// pulled this number out of thin air(-ish). Let's consider a cluster with 50k +// ranges, with each range taking ~200ms. We're being somewhat conservative with +// the duration, but in a wide-area cluster with large hops between the manager +// and the replicas, it could be true. Here's how long it'll take for various +// block sizes: +// +// page size of 1 ~ 2h 46m +// page size of 50 ~ 3m 20s +// page size of 200 ~ 50s +const defaultPageSize = 200 + +func truncatedStateMigration(ctx context.Context, h *Helper) error { + return h.UntilClusterStable(ctx, func() error { + var batchIdx, numMigratedRanges int + init := func() { batchIdx, numMigratedRanges = 1, 0 } + if err := h.IterateRangeDescriptors(ctx, defaultPageSize, init, func(descriptors ...roachpb.RangeDescriptor) error { + for _, desc := range descriptors { + // NB: This is a bit of a wart. We want to reach the first range, + // but we can't address the (local) StartKey. However, keys.LocalMax + // is on r1, so we'll just use that instead to target r1. + start, end := desc.StartKey, desc.EndKey + if bytes.Compare(desc.StartKey, keys.LocalMax) < 0 { + start, _ = keys.Addr(keys.LocalMax) + } + if err := h.DB().Migrate(ctx, start, end, h.ClusterVersion().Version); err != nil { + return err + } + } + + // TODO(irfansharif): Instead of logging this to the debug log, we + // should be leveraging our jobs infrastructure for observability. + // See #58183. + numMigratedRanges += len(descriptors) + log.Infof(ctx, "[batch %d/??] migrated %d ranges", batchIdx, numMigratedRanges) + batchIdx++ + + return nil + }); err != nil { + return err + } + + log.Infof(ctx, "[batch %d/%d] migrated %d ranges", batchIdx, batchIdx, numMigratedRanges) + + // Make sure that all stores have synced. Given we're a below-raft + // migrations, this ensures that the applied state is flushed to disk. + req := &serverpb.SyncAllEnginesRequest{} + return h.ForEveryNode(ctx, "sync-engines", func(ctx context.Context, client serverpb.MigrationClient) error { + _, err := client.SyncAllEngines(ctx, req) + return err + }) + }) +} + +func postTruncatedStateMigration(ctx context.Context, h *Helper) error { + // Purge all replicas that haven't been migrated to use the unreplicated + // truncated state and the range applied state. We're sure to also durably + // persist any changes made in the same closure. Doing so in separate + // UntilClusterStable closure would run the (small) risk that a node might + // have GC-ed older replicas, restarted without syncing (thus unapplying the + // GC), and flushing all engines after. + truncStateVersion := clusterversion.ByKey(clusterversion.TruncatedAndRangeAppliedStateMigration) + op := fmt.Sprintf("purge-outdated-replicas-and-sync=%s", truncStateVersion) + err := h.UntilClusterStable(ctx, func() error { + err := h.ForEveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { + preq := &serverpb.PurgeOutdatedReplicasRequest{Version: &truncStateVersion} + _, err := client.PurgeOutdatedReplicas(ctx, preq) + if err != nil { + return err + } + + freq := &serverpb.SyncAllEnginesRequest{} + _, err = client.SyncAllEngines(ctx, freq) + return err + }) + return err + }) + + return err +} + +// TestingRegisterMigrationInterceptor is used in tests to register an +// interceptor for a version migration. +// +// TODO(irfansharif): This is a gross anti-pattern, we're letting tests mutate +// global state. This should instead be a testing knob that the migration +// manager checks when search for attached migrations. +func TestingRegisterMigrationInterceptor( + cv clusterversion.ClusterVersion, fn migrationFn, +) (unregister func()) { + registry[cv] = Migration{cv: cv, fn: fn} + return func() { delete(registry, cv) } +} diff --git a/pkg/migration/migrations_test.go b/pkg/migration/migrations_test.go new file mode 100644 index 000000000000..d099ed1376eb --- /dev/null +++ b/pkg/migration/migrations_test.go @@ -0,0 +1,225 @@ +// 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" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" + "github.com/cockroachdb/cockroach/pkg/migration" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +func TestTruncatedStateMigration(t *testing.T) { + defer leaktest.AfterTest(t) + ctx := context.Background() + + for _, testCase := range []struct { + name string + typ stateloader.TruncatedStateType + }{ + {"ts=new,as=new", stateloader.TruncatedStateUnreplicated}, + {"ts=legacy,as=new", stateloader.TruncatedStateLegacyReplicated}, + {"ts=legacy,as=legacy", stateloader.TruncatedStateLegacyReplicatedAndNoAppliedKey}, + } { + t.Run(testCase.name, func(t *testing.T) { + args := base.TestClusterArgs{} + args.ServerArgs.Knobs.Store = &kvserver.StoreTestingKnobs{TruncatedStateTypeOverride: &testCase.typ} + args.ServerArgs.Knobs.Server = &server.TestingKnobs{ + // TruncatedAndRangeAppliedStateMigration is part of the + // migration that lets us stop using the legacy truncated state. + // When the active cluster version is greater than it, we assert + // against the presence of legacy truncated state and ensure + // we're using the range applied state key. In this test we'll + // start of at the version immediately preceding the migration, + // and migrate past it. + BinaryVersionOverride: clusterversion.ByKey(clusterversion.TruncatedAndRangeAppliedStateMigration - 1), + // We want to exercise manual control over the upgrade process. + DisableAutomaticVersionUpgrade: 1, + } + tc := testcluster.StartTestCluster(t, 3, args) + defer tc.Stopper().Stop(ctx) + + forAllReplicas := func(f func(*kvserver.Replica) error) error { + for i := 0; i < tc.NumServers(); i++ { + err := tc.Server(i).GetStores().(*kvserver.Stores).VisitStores(func(s *kvserver.Store) error { + var err error + s.VisitReplicas(func(repl *kvserver.Replica) (wantMore bool) { + err = f(repl) + return err == nil + }) + return err + }) + if err != nil { + return err + } + } + return nil + } + + getLegacyRanges := func() []string { + t.Helper() + var out []string + require.NoError(t, forAllReplicas(func(repl *kvserver.Replica) error { + sl := stateloader.Make(repl.RangeID) + + _, legacy, err := sl.LoadRaftTruncatedState(ctx, repl.Engine()) + if err != nil { + return err + } + if legacy { + // We're using the legacy truncated state, record ourselves. + out = append(out, fmt.Sprintf("ts(r%d)", repl.RangeID)) + } + + as, err := sl.LoadRangeAppliedState(ctx, repl.Engine()) + if err != nil { + return err + } + if as == nil { + // We're not using the new applied state key, record ourselves. + out = append(out, fmt.Sprintf("as(r%d)", repl.RangeID)) + } + return nil + })) + return out + } + + legacyRanges := getLegacyRanges() + switch testCase.typ { + case stateloader.TruncatedStateUnreplicated: + if len(legacyRanges) != 0 { + t.Fatalf("expected no ranges with legacy keys if bootstrapped with unreplicated truncated state, got: %v", legacyRanges) + } + case stateloader.TruncatedStateLegacyReplicated, stateloader.TruncatedStateLegacyReplicatedAndNoAppliedKey: + if len(legacyRanges) == 0 { + t.Fatalf("expected ranges with legacy keys if bootstrapped with replicated truncated state, got none") + } + } + + // NB: we'll never spot a legacy applied state here. This is + // because that migration is so aggressive that it has already + // happened as part of the initial up-replication. + t.Logf("ranges with legacy keys before migration: %v", legacyRanges) + + _, err := tc.Conns[0].ExecContext(ctx, `SET CLUSTER SETTING version = $1`, + clusterversion.ByKey(clusterversion.TruncatedAndRangeAppliedStateMigration+1).String()) + require.NoError(t, err) + require.Zero(t, getLegacyRanges()) + + require.NoError(t, forAllReplicas(func(repl *kvserver.Replica) error { + truncStateVersion := clusterversion.ByKey(clusterversion.TruncatedAndRangeAppliedStateMigration) + if repl.Version().Less(truncStateVersion) { + return errors.Newf("unexpected version %s", repl.Version()) + } + return nil + })) + }) + } +} + +func TestMigrateUpdatesReplicaVersion(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // We're going to be migrating from startCV to endCV. + startCV := clusterversion.ClusterVersion{Version: roachpb.Version{Major: 41}} + endCV := clusterversion.ClusterVersion{Version: roachpb.Version{Major: 42}} + + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Settings: cluster.MakeTestingClusterSettingsWithVersions(endCV.Version, startCV.Version, false), + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + BinaryVersionOverride: startCV.Version, + DisableAutomaticVersionUpgrade: 1, + }, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + // We'll take a specific range, still running at startCV, generate an + // outgoing snapshot and then suspend it temporarily. We'll then bump the + // cluster version on all the stores, as part of the migration process, and + // then resume the snapshot process. Seeing as how the snapshot was + // generated pre-version bump, off of a version of the range that hadn't + // observed the migration corresponding to the latest cluster version, we + // expect the store to reject it. + + key := tc.ScratchRange(t) + require.NoError(t, tc.WaitForSplitAndInitialization(key)) + desc, err := tc.LookupRange(key) + require.NoError(t, err) + rangeID := desc.RangeID + + // Enqueue the replica in the raftsnapshot queue. We use SucceedsSoon + // because it may take a bit for raft to figure out that we need to be + // generating a snapshot. + store := tc.GetFirstStoreFromServer(t, 0) + repl, err := store.GetReplica(rangeID) + require.NoError(t, err) + + if got := repl.Version(); got != startCV.Version { + t.Fatalf("got replica version %s, expected %s", got, startCV.Version) + } + + // Register the below raft migration. + unregisterKVMigration := batcheval.TestingRegisterMigrationInterceptor(endCV.Version, func() {}) + defer unregisterKVMigration() + + // Register the top-level migration. + unregister := migration.TestingRegisterMigrationInterceptor(endCV, func(ctx context.Context, h *migration.Helper) error { + return h.DB().Migrate(ctx, desc.StartKey, desc.EndKey, h.ClusterVersion().Version) + }) + defer unregister() + + // Wait until all nodes have are considered live. + nl := tc.Server(0).NodeLiveness().(*liveness.NodeLiveness) + testutils.SucceedsSoon(t, func() error { + for _, s := range tc.Servers { + id := s.NodeID() + live, err := nl.IsLive(id) + if err != nil { + return err + } + if !live { + return errors.Newf("n%s not live yet", id) + } + } + return nil + }) + + // Kick off the migration process. + _, err = tc.Conns[0].ExecContext(ctx, `SET CLUSTER SETTING version = $1`, endCV.String()) + require.NoError(t, err) + + if got := repl.Version(); got != endCV.Version { + t.Fatalf("got replica version %s, expected %s", got, endCV.Version) + } +} diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 039a88cc95c4..2ce4cc2e40c9 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 @@ -1317,6 +1326,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 a25ebd0cf11f..106a4ca0b31d 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_a1ecaa7abf76de44, []int{0} + return fileDescriptor_api_4d4da7f10ee893f6, []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_a1ecaa7abf76de44, []int{1} + return fileDescriptor_api_4d4da7f10ee893f6, []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_a1ecaa7abf76de44, []int{2} + return fileDescriptor_api_4d4da7f10ee893f6, []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_a1ecaa7abf76de44, []int{3} + return fileDescriptor_api_4d4da7f10ee893f6, []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_a1ecaa7abf76de44, []int{4} + return fileDescriptor_api_4d4da7f10ee893f6, []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_a1ecaa7abf76de44, []int{5} + return fileDescriptor_api_4d4da7f10ee893f6, []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_a1ecaa7abf76de44, []int{1, 0} + return fileDescriptor_api_4d4da7f10ee893f6, []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_a1ecaa7abf76de44, []int{25, 0} + return fileDescriptor_api_4d4da7f10ee893f6, []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_a1ecaa7abf76de44, []int{0} + return fileDescriptor_api_4d4da7f10ee893f6, []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_a1ecaa7abf76de44, []int{1} + return fileDescriptor_api_4d4da7f10ee893f6, []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_a1ecaa7abf76de44, []int{2} + return fileDescriptor_api_4d4da7f10ee893f6, []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_a1ecaa7abf76de44, []int{3} + return fileDescriptor_api_4d4da7f10ee893f6, []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_a1ecaa7abf76de44, []int{4} + return fileDescriptor_api_4d4da7f10ee893f6, []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_a1ecaa7abf76de44, []int{5} + return fileDescriptor_api_4d4da7f10ee893f6, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -646,7 +646,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_a1ecaa7abf76de44, []int{6} + return fileDescriptor_api_4d4da7f10ee893f6, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -681,7 +681,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_a1ecaa7abf76de44, []int{7} + return fileDescriptor_api_4d4da7f10ee893f6, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -727,7 +727,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_a1ecaa7abf76de44, []int{8} + return fileDescriptor_api_4d4da7f10ee893f6, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -761,7 +761,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_a1ecaa7abf76de44, []int{9} + return fileDescriptor_api_4d4da7f10ee893f6, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -801,7 +801,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_a1ecaa7abf76de44, []int{10} + return fileDescriptor_api_4d4da7f10ee893f6, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -838,7 +838,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_a1ecaa7abf76de44, []int{11} + return fileDescriptor_api_4d4da7f10ee893f6, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -872,7 +872,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_a1ecaa7abf76de44, []int{12} + return fileDescriptor_api_4d4da7f10ee893f6, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -906,7 +906,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_a1ecaa7abf76de44, []int{13} + return fileDescriptor_api_4d4da7f10ee893f6, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -958,7 +958,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_a1ecaa7abf76de44, []int{14} + return fileDescriptor_api_4d4da7f10ee893f6, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -995,7 +995,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_a1ecaa7abf76de44, []int{15} + return fileDescriptor_api_4d4da7f10ee893f6, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1050,7 +1050,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_a1ecaa7abf76de44, []int{16} + return fileDescriptor_api_4d4da7f10ee893f6, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1084,7 +1084,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_a1ecaa7abf76de44, []int{17} + return fileDescriptor_api_4d4da7f10ee893f6, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1125,7 +1125,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_a1ecaa7abf76de44, []int{18} + return fileDescriptor_api_4d4da7f10ee893f6, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1159,7 +1159,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_a1ecaa7abf76de44, []int{19} + return fileDescriptor_api_4d4da7f10ee893f6, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1210,7 +1210,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_a1ecaa7abf76de44, []int{20} + return fileDescriptor_api_4d4da7f10ee893f6, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1262,7 +1262,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_a1ecaa7abf76de44, []int{21} + return fileDescriptor_api_4d4da7f10ee893f6, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1313,7 +1313,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_a1ecaa7abf76de44, []int{22} + return fileDescriptor_api_4d4da7f10ee893f6, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1365,7 +1365,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_a1ecaa7abf76de44, []int{23} + return fileDescriptor_api_4d4da7f10ee893f6, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1418,7 +1418,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_a1ecaa7abf76de44, []int{24} + return fileDescriptor_api_4d4da7f10ee893f6, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1455,7 +1455,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_a1ecaa7abf76de44, []int{25} + return fileDescriptor_api_4d4da7f10ee893f6, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1499,7 +1499,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_a1ecaa7abf76de44, []int{25, 0} + return fileDescriptor_api_4d4da7f10ee893f6, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1547,7 +1547,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_a1ecaa7abf76de44, []int{26} + return fileDescriptor_api_4d4da7f10ee893f6, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1583,7 +1583,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_a1ecaa7abf76de44, []int{27} + return fileDescriptor_api_4d4da7f10ee893f6, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1694,7 +1694,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_a1ecaa7abf76de44, []int{28} + return fileDescriptor_api_4d4da7f10ee893f6, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1740,7 +1740,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_a1ecaa7abf76de44, []int{29} + return fileDescriptor_api_4d4da7f10ee893f6, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1801,7 +1801,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_a1ecaa7abf76de44, []int{30} + return fileDescriptor_api_4d4da7f10ee893f6, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1836,7 +1836,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_a1ecaa7abf76de44, []int{31} + return fileDescriptor_api_4d4da7f10ee893f6, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1875,7 +1875,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_a1ecaa7abf76de44, []int{32} + return fileDescriptor_api_4d4da7f10ee893f6, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1910,7 +1910,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_a1ecaa7abf76de44, []int{33} + return fileDescriptor_api_4d4da7f10ee893f6, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1953,7 +1953,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_a1ecaa7abf76de44, []int{34} + return fileDescriptor_api_4d4da7f10ee893f6, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1988,7 +1988,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_a1ecaa7abf76de44, []int{35} + return fileDescriptor_api_4d4da7f10ee893f6, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2026,7 +2026,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_a1ecaa7abf76de44, []int{36} + return fileDescriptor_api_4d4da7f10ee893f6, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2059,7 +2059,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_a1ecaa7abf76de44, []int{37} + return fileDescriptor_api_4d4da7f10ee893f6, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2094,7 +2094,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_a1ecaa7abf76de44, []int{38} + return fileDescriptor_api_4d4da7f10ee893f6, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2152,7 +2152,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_a1ecaa7abf76de44, []int{39} + return fileDescriptor_api_4d4da7f10ee893f6, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2187,7 +2187,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_a1ecaa7abf76de44, []int{40} + return fileDescriptor_api_4d4da7f10ee893f6, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2224,7 +2224,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_a1ecaa7abf76de44, []int{41} + return fileDescriptor_api_4d4da7f10ee893f6, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2257,7 +2257,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_a1ecaa7abf76de44, []int{42} + return fileDescriptor_api_4d4da7f10ee893f6, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2296,7 +2296,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_a1ecaa7abf76de44, []int{43} + return fileDescriptor_api_4d4da7f10ee893f6, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2333,7 +2333,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_a1ecaa7abf76de44, []int{44} + return fileDescriptor_api_4d4da7f10ee893f6, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2371,7 +2371,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_a1ecaa7abf76de44, []int{45} + return fileDescriptor_api_4d4da7f10ee893f6, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2405,7 +2405,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_a1ecaa7abf76de44, []int{45, 0} + return fileDescriptor_api_4d4da7f10ee893f6, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2439,7 +2439,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_a1ecaa7abf76de44, []int{46} + return fileDescriptor_api_4d4da7f10ee893f6, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2508,7 +2508,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_a1ecaa7abf76de44, []int{47} + return fileDescriptor_api_4d4da7f10ee893f6, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2551,7 +2551,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_a1ecaa7abf76de44, []int{48} + return fileDescriptor_api_4d4da7f10ee893f6, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2598,7 +2598,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_a1ecaa7abf76de44, []int{49} + return fileDescriptor_api_4d4da7f10ee893f6, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2634,7 +2634,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_a1ecaa7abf76de44, []int{50} + return fileDescriptor_api_4d4da7f10ee893f6, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2678,7 +2678,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_a1ecaa7abf76de44, []int{51} + return fileDescriptor_api_4d4da7f10ee893f6, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2722,7 +2722,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_a1ecaa7abf76de44, []int{52} + return fileDescriptor_api_4d4da7f10ee893f6, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2782,7 +2782,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_a1ecaa7abf76de44, []int{53} + return fileDescriptor_api_4d4da7f10ee893f6, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2818,7 +2818,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_a1ecaa7abf76de44, []int{54} + return fileDescriptor_api_4d4da7f10ee893f6, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2865,7 +2865,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_a1ecaa7abf76de44, []int{55} + return fileDescriptor_api_4d4da7f10ee893f6, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2900,7 +2900,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_a1ecaa7abf76de44, []int{56} + return fileDescriptor_api_4d4da7f10ee893f6, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2951,7 +2951,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_a1ecaa7abf76de44, []int{57} + return fileDescriptor_api_4d4da7f10ee893f6, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2986,7 +2986,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_a1ecaa7abf76de44, []int{58} + return fileDescriptor_api_4d4da7f10ee893f6, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3023,7 +3023,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_a1ecaa7abf76de44, []int{59} + return fileDescriptor_api_4d4da7f10ee893f6, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3057,7 +3057,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_a1ecaa7abf76de44, []int{60} + return fileDescriptor_api_4d4da7f10ee893f6, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3102,7 +3102,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_a1ecaa7abf76de44, []int{61} + return fileDescriptor_api_4d4da7f10ee893f6, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3136,7 +3136,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_a1ecaa7abf76de44, []int{62} + return fileDescriptor_api_4d4da7f10ee893f6, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3180,7 +3180,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_a1ecaa7abf76de44, []int{63} + return fileDescriptor_api_4d4da7f10ee893f6, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3229,7 +3229,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_a1ecaa7abf76de44, []int{64} + return fileDescriptor_api_4d4da7f10ee893f6, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3266,7 +3266,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_a1ecaa7abf76de44, []int{65} + return fileDescriptor_api_4d4da7f10ee893f6, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3303,7 +3303,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_a1ecaa7abf76de44, []int{66} + return fileDescriptor_api_4d4da7f10ee893f6, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3338,7 +3338,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_a1ecaa7abf76de44, []int{67} + return fileDescriptor_api_4d4da7f10ee893f6, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3393,7 +3393,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_a1ecaa7abf76de44, []int{68} + return fileDescriptor_api_4d4da7f10ee893f6, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3430,7 +3430,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_a1ecaa7abf76de44, []int{69} + return fileDescriptor_api_4d4da7f10ee893f6, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3470,7 +3470,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_a1ecaa7abf76de44, []int{70} + return fileDescriptor_api_4d4da7f10ee893f6, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3504,7 +3504,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_a1ecaa7abf76de44, []int{70, 0} + return fileDescriptor_api_4d4da7f10ee893f6, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3537,7 +3537,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_a1ecaa7abf76de44, []int{70, 1} + return fileDescriptor_api_4d4da7f10ee893f6, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3579,7 +3579,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_a1ecaa7abf76de44, []int{70, 2} + return fileDescriptor_api_4d4da7f10ee893f6, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3618,7 +3618,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_a1ecaa7abf76de44, []int{70, 3} + return fileDescriptor_api_4d4da7f10ee893f6, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3654,7 +3654,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_a1ecaa7abf76de44, []int{70, 4} + return fileDescriptor_api_4d4da7f10ee893f6, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3693,7 +3693,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_a1ecaa7abf76de44, []int{70, 5} + return fileDescriptor_api_4d4da7f10ee893f6, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3735,7 +3735,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_a1ecaa7abf76de44, []int{70, 6} + return fileDescriptor_api_4d4da7f10ee893f6, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3775,7 +3775,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_a1ecaa7abf76de44, []int{71} + return fileDescriptor_api_4d4da7f10ee893f6, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3809,7 +3809,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_a1ecaa7abf76de44, []int{72} + return fileDescriptor_api_4d4da7f10ee893f6, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3845,7 +3845,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_a1ecaa7abf76de44, []int{73} + return fileDescriptor_api_4d4da7f10ee893f6, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3915,7 +3915,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_a1ecaa7abf76de44, []int{74} + return fileDescriptor_api_4d4da7f10ee893f6, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3967,7 +3967,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_a1ecaa7abf76de44, []int{75} + return fileDescriptor_api_4d4da7f10ee893f6, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4003,7 +4003,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_a1ecaa7abf76de44, []int{76} + return fileDescriptor_api_4d4da7f10ee893f6, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4043,7 +4043,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_a1ecaa7abf76de44, []int{76, 0} + return fileDescriptor_api_4d4da7f10ee893f6, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4094,7 +4094,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_a1ecaa7abf76de44, []int{77} + return fileDescriptor_api_4d4da7f10ee893f6, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4129,7 +4129,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_a1ecaa7abf76de44, []int{77, 0} + return fileDescriptor_api_4d4da7f10ee893f6, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4165,7 +4165,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_a1ecaa7abf76de44, []int{77, 1} + return fileDescriptor_api_4d4da7f10ee893f6, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4200,7 +4200,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_a1ecaa7abf76de44, []int{78} + return fileDescriptor_api_4d4da7f10ee893f6, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4238,7 +4238,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_a1ecaa7abf76de44, []int{79} + return fileDescriptor_api_4d4da7f10ee893f6, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4275,7 +4275,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_a1ecaa7abf76de44, []int{80} + return fileDescriptor_api_4d4da7f10ee893f6, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4308,7 +4308,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_a1ecaa7abf76de44, []int{80, 0} + return fileDescriptor_api_4d4da7f10ee893f6, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4353,7 +4353,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_a1ecaa7abf76de44, []int{81} + return fileDescriptor_api_4d4da7f10ee893f6, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4391,7 +4391,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_a1ecaa7abf76de44, []int{82} + return fileDescriptor_api_4d4da7f10ee893f6, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4444,7 +4444,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_a1ecaa7abf76de44, []int{83} + return fileDescriptor_api_4d4da7f10ee893f6, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4478,7 +4478,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_a1ecaa7abf76de44, []int{84} + return fileDescriptor_api_4d4da7f10ee893f6, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4522,7 +4522,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_a1ecaa7abf76de44, []int{85} + return fileDescriptor_api_4d4da7f10ee893f6, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4556,7 +4556,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_a1ecaa7abf76de44, []int{86} + return fileDescriptor_api_4d4da7f10ee893f6, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4595,7 +4595,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_a1ecaa7abf76de44, []int{87} + return fileDescriptor_api_4d4da7f10ee893f6, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4629,7 +4629,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_a1ecaa7abf76de44, []int{88} + return fileDescriptor_api_4d4da7f10ee893f6, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4678,7 +4678,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_a1ecaa7abf76de44, []int{89} + return fileDescriptor_api_4d4da7f10ee893f6, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4727,7 +4727,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_a1ecaa7abf76de44, []int{90} + return fileDescriptor_api_4d4da7f10ee893f6, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4762,7 +4762,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_a1ecaa7abf76de44, []int{91} + return fileDescriptor_api_4d4da7f10ee893f6, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4804,7 +4804,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_a1ecaa7abf76de44, []int{92} + return fileDescriptor_api_4d4da7f10ee893f6, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4829,6 +4829,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_4d4da7f10ee893f6, []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_4d4da7f10ee893f6, []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. // @@ -4880,6 +4957,7 @@ type RequestUnion struct { // *RequestUnion_Subsume // *RequestUnion_RangeStats // *RequestUnion_AdminVerifyProtectedTimestamp + // *RequestUnion_Migrate Value isRequestUnion_Value `protobuf_oneof:"value"` } @@ -4887,7 +4965,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_a1ecaa7abf76de44, []int{93} + return fileDescriptor_api_4d4da7f10ee893f6, []int{95} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5050,6 +5128,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() {} @@ -5095,6 +5176,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 { @@ -5411,6 +5493,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{}{ @@ -5458,6 +5547,7 @@ func (*RequestUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) (*RequestUnion_Subsume)(nil), (*RequestUnion_RangeStats)(nil), (*RequestUnion_AdminVerifyProtectedTimestamp)(nil), + (*RequestUnion_Migrate)(nil), } } @@ -5685,6 +5775,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) @@ -6047,6 +6142,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 } @@ -6276,6 +6379,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)) @@ -6330,6 +6438,7 @@ type ResponseUnion struct { // *ResponseUnion_Subsume // *ResponseUnion_RangeStats // *ResponseUnion_AdminVerifyProtectedTimestamp + // *ResponseUnion_Migrate Value isResponseUnion_Value `protobuf_oneof:"value"` } @@ -6337,7 +6446,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_a1ecaa7abf76de44, []int{94} + return fileDescriptor_api_4d4da7f10ee893f6, []int{96} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6497,6 +6606,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() {} @@ -6541,6 +6653,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 { @@ -6850,6 +6963,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{}{ @@ -6896,6 +7016,7 @@ func (*ResponseUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) (*ResponseUnion_Subsume)(nil), (*ResponseUnion_RangeStats)(nil), (*ResponseUnion_AdminVerifyProtectedTimestamp)(nil), + (*ResponseUnion_Migrate)(nil), } } @@ -7118,6 +7239,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) @@ -7472,6 +7598,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 } @@ -7696,6 +7830,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)) @@ -7872,7 +8011,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_a1ecaa7abf76de44, []int{95} + return fileDescriptor_api_4d4da7f10ee893f6, []int{97} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7910,7 +8049,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_a1ecaa7abf76de44, []int{96} + return fileDescriptor_api_4d4da7f10ee893f6, []int{98} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7946,7 +8085,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{97} + return fileDescriptor_api_4d4da7f10ee893f6, []int{99} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7983,7 +8122,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{98} + return fileDescriptor_api_4d4da7f10ee893f6, []int{100} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8062,7 +8201,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_a1ecaa7abf76de44, []int{98, 0} + return fileDescriptor_api_4d4da7f10ee893f6, []int{100, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8100,7 +8239,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_a1ecaa7abf76de44, []int{99} + return fileDescriptor_api_4d4da7f10ee893f6, []int{101} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8138,7 +8277,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_a1ecaa7abf76de44, []int{100} + return fileDescriptor_api_4d4da7f10ee893f6, []int{102} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8177,7 +8316,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_a1ecaa7abf76de44, []int{101} + return fileDescriptor_api_4d4da7f10ee893f6, []int{103} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8218,7 +8357,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_a1ecaa7abf76de44, []int{102} + return fileDescriptor_api_4d4da7f10ee893f6, []int{104} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8259,7 +8398,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_a1ecaa7abf76de44, []int{103} + return fileDescriptor_api_4d4da7f10ee893f6, []int{105} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8296,7 +8435,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_a1ecaa7abf76de44, []int{104} + return fileDescriptor_api_4d4da7f10ee893f6, []int{106} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8333,7 +8472,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_a1ecaa7abf76de44, []int{105} + return fileDescriptor_api_4d4da7f10ee893f6, []int{107} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8377,7 +8516,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_a1ecaa7abf76de44, []int{106} + return fileDescriptor_api_4d4da7f10ee893f6, []int{108} } func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8409,7 +8548,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_a1ecaa7abf76de44, []int{107} + return fileDescriptor_api_4d4da7f10ee893f6, []int{109} } func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8450,7 +8589,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_a1ecaa7abf76de44, []int{108} + return fileDescriptor_api_4d4da7f10ee893f6, []int{110} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8490,7 +8629,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_a1ecaa7abf76de44, []int{109} + return fileDescriptor_api_4d4da7f10ee893f6, []int{111} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8526,7 +8665,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_a1ecaa7abf76de44, []int{110} + return fileDescriptor_api_4d4da7f10ee893f6, []int{112} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8565,7 +8704,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_a1ecaa7abf76de44, []int{111} + return fileDescriptor_api_4d4da7f10ee893f6, []int{113} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8608,7 +8747,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_a1ecaa7abf76de44, []int{112} + return fileDescriptor_api_4d4da7f10ee893f6, []int{114} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8742,6 +8881,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") @@ -13467,6 +13608,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) @@ -13483,11 +13684,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 } @@ -13498,11 +13699,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 } @@ -13512,11 +13713,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 } @@ -13526,11 +13727,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 } @@ -13540,11 +13741,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 } @@ -13554,11 +13755,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 } @@ -13568,11 +13769,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 } @@ -13582,11 +13783,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 } @@ -13596,11 +13797,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 } @@ -13610,11 +13811,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 } @@ -13624,11 +13825,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 } @@ -13638,11 +13839,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 } @@ -13652,11 +13853,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 } @@ -13666,11 +13867,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 } @@ -13682,11 +13883,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 } @@ -13698,11 +13899,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 } @@ -13714,11 +13915,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 } @@ -13730,11 +13931,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 } @@ -13746,11 +13947,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 } @@ -13762,11 +13963,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 } @@ -13778,11 +13979,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 } @@ -13794,11 +13995,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 } @@ -13810,11 +14011,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 } @@ -13826,11 +14027,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 } @@ -13842,11 +14043,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 } @@ -13858,11 +14059,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 } @@ -13874,11 +14075,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 } @@ -13890,11 +14091,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 } @@ -13906,11 +14107,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 } @@ -13922,11 +14123,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 } @@ -13938,11 +14139,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 } @@ -13954,11 +14155,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 } @@ -13970,11 +14171,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 } @@ -13986,11 +14187,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 } @@ -14002,11 +14203,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 } @@ -14018,11 +14219,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 } @@ -14034,11 +14235,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 } @@ -14050,11 +14251,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 } @@ -14066,11 +14267,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 } @@ -14082,11 +14283,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 } @@ -14098,11 +14299,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 } @@ -14114,11 +14315,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 } @@ -14130,11 +14331,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 } @@ -14146,11 +14347,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 } @@ -14162,11 +14363,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 } @@ -14186,11 +14403,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 } @@ -14201,11 +14418,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 } @@ -14215,11 +14432,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 } @@ -14229,11 +14446,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 } @@ -14243,11 +14460,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 } @@ -14257,11 +14474,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 } @@ -14271,11 +14488,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 } @@ -14285,11 +14502,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 } @@ -14299,11 +14516,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 } @@ -14313,11 +14530,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 } @@ -14327,11 +14544,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 } @@ -14341,11 +14558,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 } @@ -14355,11 +14572,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 } @@ -14369,11 +14586,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 } @@ -14385,11 +14602,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 } @@ -14401,11 +14618,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 } @@ -14417,11 +14634,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 } @@ -14433,11 +14650,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 } @@ -14449,11 +14666,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 } @@ -14465,11 +14682,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 } @@ -14481,11 +14698,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 } @@ -14497,11 +14714,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 } @@ -14513,11 +14730,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 } @@ -14529,11 +14746,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 } @@ -14545,11 +14762,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 } @@ -14561,11 +14778,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 } @@ -14577,11 +14794,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 } @@ -14593,11 +14810,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 } @@ -14609,11 +14826,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 } @@ -14625,11 +14842,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 } @@ -14641,11 +14858,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 } @@ -14657,11 +14874,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 } @@ -14673,11 +14890,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 } @@ -14689,11 +14906,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 } @@ -14705,11 +14922,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 } @@ -14721,11 +14938,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 } @@ -14737,11 +14954,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 } @@ -14753,11 +14970,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 } @@ -14769,11 +14986,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 } @@ -14785,11 +15002,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 } @@ -14801,11 +15018,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 } @@ -14817,11 +15034,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 } @@ -14833,11 +15050,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 } @@ -14849,11 +15066,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 } @@ -14875,19 +15108,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++ @@ -14903,11 +15136,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 @@ -14977,11 +15210,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 @@ -15039,11 +15272,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 @@ -15077,11 +15310,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 @@ -15116,38 +15349,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 @@ -15262,11 +15495,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 } @@ -15289,19 +15522,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++ @@ -15339,19 +15572,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 } @@ -15373,19 +15606,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 } @@ -15407,11 +15640,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 } @@ -15434,31 +15667,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 } @@ -15561,11 +15794,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++ @@ -15576,11 +15809,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 } @@ -15604,11 +15837,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 } @@ -15648,11 +15881,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 } @@ -15681,19 +15914,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 } @@ -17591,6 +17824,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 @@ -18131,6 +18388,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 @@ -18659,6 +18928,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 @@ -32988,6 +33269,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 @@ -34425,6 +34896,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:]) @@ -35851,6 +36354,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:]) @@ -38545,516 +39080,520 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_a1ecaa7abf76de44) } - -var fileDescriptor_api_a1ecaa7abf76de44 = []byte{ - // 8127 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, 0x06, 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, 0xc0, 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, 0x11, 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, 0x6b, 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, 0x69, 0x02, - 0x2e, 0x55, 0x4c, 0x43, 0xd3, 0xc9, 0x94, 0x54, 0xda, 0x5f, 0x0b, 0xad, 0xac, 0x43, 0x60, 0xf2, - 0xc9, 0xf8, 0xb8, 0x3b, 0x66, 0x1f, 0x23, 0x9f, 0xab, 0x7a, 0xdc, 0xa5, 0xb4, 0x68, 0x4d, 0xa2, - 0x6f, 0xc2, 0x15, 0xa5, 0xdd, 0x36, 0x8f, 0x64, 0x7d, 0x4f, 0xd6, 0x4c, 0x6c, 0xcb, 0x86, 0xe9, - 0xc8, 0xf8, 0x58, 0xb7, 0x1d, 0x6a, 0x4a, 0xd2, 0xd2, 0x1c, 0xbd, 0x5d, 0xdf, 0x5b, 0x33, 0xb1, - 0xbd, 0x65, 0x3a, 0x55, 0x72, 0x8b, 0xcc, 0x53, 0x52, 0x19, 0x36, 0x4f, 0xa7, 0x88, 0xf9, 0x95, - 0xd2, 0xf8, 0xb8, 0x4b, 0xe7, 0x69, 0xa0, 0x2b, 0xa7, 0x83, 0x5d, 0x29, 0xfe, 0x12, 0x5c, 0xee, - 0xd7, 0x6f, 0x9c, 0xfd, 0xf7, 0x27, 0x02, 0xe4, 0xeb, 0x86, 0xee, 0x7c, 0x2d, 0x3a, 0xce, 0x53, - 0x76, 0x32, 0xa8, 0xec, 0xfb, 0x50, 0xdc, 0x53, 0xf4, 0xf6, 0x73, 0xa3, 0x69, 0x76, 0x76, 0x6d, - 0xc7, 0x34, 0xb0, 0xcd, 0x7b, 0x63, 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, 0x97, 0x64, 0xc9, 0xc0, - 0x47, 0xb2, 0xdf, 0x47, 0x64, 0xc9, 0xc0, 0x47, 0xcc, 0xbc, 0x35, 0x20, 0xb7, 0x86, 0xdb, 0xd8, - 0xc1, 0x71, 0x5a, 0xfd, 0x1d, 0xc8, 0xbb, 0x42, 0xe3, 0xec, 0x98, 0xbf, 0x27, 0x00, 0xe2, 0x72, - 0xc9, 0x2a, 0x1b, 0x67, 0xdf, 0x2c, 0x12, 0xd7, 0xc2, 0xe9, 0x59, 0x06, 0xf3, 0x11, 0xd8, 0x98, - 0x04, 0x46, 0xa2, 0x6e, 0x82, 0x3f, 0x65, 0x53, 0xc1, 0x29, 0xcb, 0xdd, 0x9b, 0x23, 0x98, 0x0b, - 0x55, 0x2c, 0xde, 0xee, 0x4b, 0xd1, 0x3a, 0x25, 0x96, 0x92, 0x41, 0x1f, 0x8e, 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, 0xa7, 0xbb, 0xbc, 0xdf, 0x56, 0x97, 0x9b, 0xae, 0x0f, - 0xcc, 0x67, 0xb7, 0xc7, 0x24, 0x7e, 0x06, 0x28, 0x58, 0xb3, 0x38, 0x07, 0xc2, 0x3f, 0x14, 0x00, - 0x49, 0xf8, 0x10, 0x5b, 0x4e, 0xec, 0xcd, 0x5e, 0x83, 0xac, 0xa3, 0x58, 0x2d, 0xec, 0xc8, 0xc4, - 0xbb, 0xbf, 0x48, 0xcb, 0x81, 0xf1, 0x11, 0xb2, 0xf8, 0x39, 0xcc, 0x85, 0xea, 0x17, 0x67, 0xe3, - 0xff, 0xb7, 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, 0x9e, 0xb5, 0xaa, 0x18, 0x4f, - 0x68, 0x21, 0x09, 0x6c, 0xef, 0x37, 0x7a, 0x01, 0xd9, 0x03, 0x7c, 0x22, 0x73, 0x94, 0x46, 0xd7, - 0xc5, 0xfc, 0xc3, 0xf7, 0x03, 0xfc, 0x07, 0x87, 0xcb, 0x2e, 0xb8, 0x5b, 0x0e, 0x80, 0xbb, 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, 0x57, 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, 0x43, 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, 0x13, 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, 0x37, 0x3a, 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, 0xb8, 0xbb, 0xed, 0xf0, 0x31, 0xf2, 0x70, 0x98, 0x0a, 0x22, 0x6a, 0x40, - 0xf1, 0x7b, 0xdb, 0xe1, 0xd5, 0xe6, 0x72, 0x16, 0xfe, 0xbd, 0x00, 0x53, 0xec, 0x06, 0x7a, 0x00, - 0x69, 0xbe, 0xd1, 0xa0, 0xd1, 0x3a, 0x26, 0xcb, 0x97, 0xcf, 0x4e, 0x17, 0xa7, 0xd9, 0xde, 0xc1, - 0xda, 0x97, 0xfe, 0x4f, 0x69, 0x9a, 0x96, 0xab, 0x6b, 0xa4, 0xb7, 0x6c, 0x47, 0xb1, 0x1c, 0xba, - 0x9d, 0x93, 0x60, 0x78, 0x82, 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, 0xcf, 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, 0xbf, 0x14, 0xe0, 0x72, 0xff, 0x63, - 0xe3, 0x1c, 0x84, 0x2f, 0x20, 0xab, 0x68, 0x1a, 0xd6, 0x64, 0x0d, 0xb7, 0x1d, 0x85, 0x3b, 0x23, - 0xf7, 0x03, 0x92, 0xf8, 0x26, 0xdc, 0xb2, 0xb7, 0x09, 0xb7, 0xf9, 0xb2, 0x52, 0xa1, 0x15, 0x59, - 0x23, 0x1c, 0xae, 0xf9, 0xa1, 0x42, 0x28, 0x45, 0xfc, 0x5f, 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, 0x7f, 0x77, 0x23, - 0xea, 0x59, 0xe7, 0x1c, 0x15, 0xca, 0xd0, 0x64, 0xe5, 0xa5, 0x4b, 0x7a, 0x14, 0x19, 0x7d, 0x02, - 0x40, 0x96, 0x22, 0xba, 0x99, 0x68, 0x73, 0x7b, 0x34, 0x6c, 0x37, 0xd1, 0x35, 0x41, 0x84, 0x81, - 0x5c, 0xdb, 0x68, 0x85, 0x20, 0x86, 0xd7, 0x3d, 0xdd, 0xc2, 0xf2, 0x83, 0xae, 0x4a, 0x21, 0x7e, - 0xba, 0x9c, 0x3f, 0x3b, 0x5d, 0x04, 0x89, 0x91, 0x1f, 0x6c, 0x57, 0x08, 0x82, 0x60, 0xbf, 0xbb, - 0x2a, 0x7a, 0x05, 0xf7, 0x02, 0x3b, 0x15, 0x64, 0xe5, 0xe5, 0xcd, 0x52, 0x1c, 0x79, 0x5f, 0x6f, - 0xed, 0x63, 0x4b, 0xf6, 0x36, 0x94, 0xe9, 0xce, 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, 0x78, 0xb5, 0x57, 0xa4, 0x24, 0x57, 0x4d, 0x5e, 0x37, 0x9e, 0x50, - 0x7e, 0x4a, 0xb4, 0xd7, 0x53, 0xe9, 0xe9, 0x62, 0x5a, 0xfc, 0x6f, 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, 0x4b, 0x02, 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, 0x7b, 0x94, - 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, 0x55, 0x92, 0x2b, 0x81, 0x3d, 0x83, 0x17, 0x8f, 0xd5, - 0x47, 0xf9, 0x3c, 0xf4, 0x1a, 0x2b, 0xd8, 0xf0, 0xc4, 0x05, 0x1a, 0x1e, 0xd8, 0x4b, 0xf7, 0xa9, - 0xe8, 0x33, 0x08, 0xec, 0x96, 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, - 0xdf, 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, 0x21, 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, 0x50, 0x83, 0x10, - 0x11, 0xd4, 0x40, 0x1e, 0xe3, 0xb9, 0x71, 0x89, 0x8b, 0x3c, 0x26, 0xe0, 0xbf, 0x4d, 0x16, 0xa7, - 0xc4, 0x17, 0x00, 0xa4, 0x39, 0x71, 0x76, 0xc9, 0xdf, 0x4a, 0x42, 0x7e, 0xbb, 0x67, 0xef, 0xc7, - 0x3c, 0xfa, 0x2a, 0x00, 0xdd, 0x9e, 0x4d, 0xf1, 0xc1, 0xb1, 0xc1, 0xdb, 0x7c, 0x4e, 0xbc, 0x84, - 0xdb, 0x68, 0xc6, 0xd7, 0x3c, 0x36, 0x50, 0x8d, 0x0b, 0xc1, 0xb2, 0x1f, 0x74, 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, 0xaa, 0xfb, 0xda, 0x94, 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, 0x91, 0xba, 0x2b, 0xaa, 0x79, 0x48, - 0x7b, 0x26, 0xce, 0x91, 0xf2, 0x98, 0x85, 0xd4, 0x24, 0x2e, 0xda, 0xbb, 0x34, 0xba, 0xe6, 0x01, - 0xcc, 0xeb, 0x1d, 0x62, 0xc7, 0x75, 0xa7, 0x7d, 0xc2, 0x31, 0x95, 0x83, 0xdd, 0xf7, 0xb3, 0x73, - 0xfe, 0xbd, 0x8a, 0x7b, 0x4b, 0xfc, 0xc7, 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, 0xb8, 0x8e, 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, 0xf5, 0x74, 0xf1, 0xd1, 0x58, 0xd1, 0x5a, 0x34, 0x32, 0xad, 0xd7, 0xd3, 0xb5, - 0xe5, 0x9d, 0x9d, 0xfa, 0x9a, 0x54, 0xa4, 0x22, 0x5f, 0x31, 0x89, 0xcd, 0x63, 0xc3, 0x16, 0xff, - 0x49, 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, 0x9c, 0x1e, 0x8b, 0xed, 0x70, 0xc3, - 0x0e, 0x0a, 0x0e, 0xd1, 0x07, 0xa1, 0xd3, 0xb8, 0x0e, 0x5b, 0xfc, 0x23, 0x01, 0x10, 0x55, 0x54, - 0x9d, 0xed, 0xe9, 0x7f, 0x5d, 0xc6, 0xd3, 0x5d, 0x28, 0xd2, 0xc8, 0x47, 0x59, 0xdf, 0x93, 0x3b, - 0xba, 0x6d, 0xeb, 0x46, 0x8b, 0x0f, 0xa8, 0x3c, 0xa5, 0xd7, 0xf7, 0x36, 0x19, 0x55, 0xfc, 0xeb, - 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, 0x13, 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, 0xef, 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, 0x4f, 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, 0x67, 0x01, 0xe6, 0xbf, 0xb2, 0xad, 0xd8, 0xbf, 0x0a, 0x5d, - 0x89, 0x2f, 0xa1, 0x48, 0x7f, 0xd4, 0x8d, 0x3d, 0x33, 0xce, 0x4d, 0xf1, 0xef, 0x09, 0x30, 0x1b, - 0x10, 0x1c, 0xa7, 0x7f, 0xf2, 0x46, 0x7a, 0x12, 0x7f, 0x91, 0x78, 0x2c, 0xc1, 0x41, 0x1e, 0xe7, - 0x14, 0xfa, 0x83, 0x04, 0x5c, 0xae, 0xb0, 0x57, 0xc8, 0x6e, 0x4c, 0x45, 0x9c, 0x23, 0xa3, 0x04, - 0xd3, 0x87, 0xd8, 0xb2, 0x75, 0x93, 0xad, 0x9e, 0x39, 0xc9, 0xbd, 0x44, 0x0b, 0x90, 0xb6, 0x0d, - 0xa5, 0x6b, 0xef, 0x9b, 0xee, 0xbb, 0x33, 0xef, 0xda, 0x8b, 0xff, 0x98, 0x7c, 0xf3, 0xf8, 0x8f, - 0xa9, 0xd1, 0xf1, 0x1f, 0xd3, 0x3f, 0x41, 0xfc, 0x07, 0x7f, 0x51, 0xf5, 0x9f, 0x04, 0xb8, 0x32, - 0xa0, 0xb9, 0x38, 0x47, 0xcb, 0x77, 0x21, 0xab, 0x72, 0xc1, 0xc4, 0xde, 0xb2, 0xb7, 0x70, 0x75, - 0x52, 0xec, 0x0d, 0x61, 0xc7, 0xd9, 0xe9, 0x22, 0xb8, 0x55, 0xad, 0xaf, 0x71, 0xe5, 0x90, 0xdf, - 0x9a, 0xf8, 0x2b, 0x39, 0x28, 0x54, 0x8f, 0xd9, 0x0e, 0x74, 0x83, 0xad, 0xf2, 0xe8, 0x09, 0xa4, - 0xbb, 0x96, 0x79, 0xa8, 0xbb, 0xcd, 0xc8, 0x87, 0x5e, 0xfe, 0xbb, 0xcd, 0xe8, 0xe3, 0xda, 0xe6, - 0x1c, 0x92, 0xc7, 0x8b, 0x9a, 0x90, 0x79, 0x66, 0xaa, 0x4a, 0xfb, 0x89, 0xde, 0x76, 0x47, 0xfe, - 0xfb, 0xe7, 0x0b, 0x5a, 0xf6, 0x78, 0xb6, 0x15, 0x67, 0xdf, 0xed, 0x04, 0x8f, 0x88, 0xea, 0x90, - 0xae, 0x39, 0x4e, 0x97, 0xdc, 0xe4, 0xb6, 0xe3, 0xce, 0x18, 0x42, 0x09, 0x8b, 0x1b, 0x2b, 0xea, - 0xb2, 0xa3, 0x26, 0xcc, 0x3e, 0xa5, 0x27, 0x9f, 0x2a, 0x6d, 0xb3, 0xa7, 0x55, 0x4c, 0x63, 0x4f, - 0x6f, 0x71, 0xbb, 0x7b, 0x7b, 0x0c, 0x99, 0x4f, 0x2b, 0x0d, 0x69, 0x50, 0x00, 0x5a, 0x85, 0x74, - 0xe3, 0x11, 0x17, 0xc6, 0xdc, 0xb2, 0x5b, 0x63, 0x08, 0x6b, 0x3c, 0x92, 0x3c, 0x36, 0xb4, 0x0e, - 0xd9, 0xd5, 0x2f, 0x7a, 0x16, 0xe6, 0x52, 0xa6, 0x86, 0x46, 0x1e, 0xf4, 0x4b, 0xa1, 0x5c, 0x52, - 0x90, 0x19, 0x35, 0x20, 0xff, 0xca, 0xb4, 0x0e, 0xda, 0xa6, 0xe2, 0xb6, 0x70, 0x9a, 0x8a, 0xfb, - 0xc6, 0x18, 0xe2, 0x5c, 0x46, 0xa9, 0x4f, 0x04, 0xfa, 0x36, 0x14, 0x48, 0x67, 0x34, 0x95, 0xdd, - 0xb6, 0x5b, 0xc9, 0x34, 0x95, 0xfa, 0xee, 0x18, 0x52, 0x3d, 0x4e, 0xf7, 0x15, 0x48, 0x9f, 0xa8, - 0x05, 0x09, 0x72, 0xa1, 0x41, 0x80, 0x10, 0xa4, 0xba, 0xa4, 0xbf, 0x05, 0x1a, 0x1b, 0x44, 0x7f, - 0xa3, 0xf7, 0x60, 0xda, 0x30, 0x35, 0xec, 0xce, 0x90, 0x5c, 0x79, 0xfe, 0xec, 0x74, 0x71, 0x6a, - 0xcb, 0xd4, 0x98, 0x43, 0xc2, 0x7f, 0x49, 0x53, 0xa4, 0x50, 0x5d, 0x5b, 0x58, 0x82, 0x14, 0xe9, - 0x77, 0x62, 0x98, 0x76, 0x15, 0x1b, 0xef, 0x58, 0x3a, 0x97, 0xe6, 0x5e, 0x2e, 0xfc, 0x8b, 0x04, - 0x24, 0x1a, 0x8f, 0x88, 0xcb, 0xbd, 0xdb, 0x53, 0x0f, 0xb0, 0xc3, 0xef, 0xf3, 0x2b, 0xea, 0x8a, - 0x5b, 0x78, 0x4f, 0x67, 0x9e, 0x51, 0x46, 0xe2, 0x57, 0xe8, 0x1d, 0x00, 0x45, 0x55, 0xb1, 0x6d, - 0xcb, 0xee, 0x89, 0xb8, 0x8c, 0x94, 0x61, 0x94, 0x0d, 0x7c, 0x42, 0xd8, 0x6c, 0xac, 0x5a, 0xd8, - 0x71, 0x03, 0x9b, 0xd8, 0x15, 0x61, 0x73, 0x70, 0xa7, 0x2b, 0x3b, 0xe6, 0x01, 0x36, 0xe8, 0x38, - 0xc9, 0x10, 0x53, 0xd3, 0xe9, 0x36, 0x09, 0x81, 0x58, 0x49, 0x6c, 0x68, 0xbe, 0x49, 0xcb, 0x48, - 0xde, 0x35, 0x11, 0x69, 0xe1, 0x96, 0xce, 0x8f, 0x76, 0x65, 0x24, 0x7e, 0x45, 0xb4, 0xa4, 0xf4, - 0x9c, 0x7d, 0xda, 0x13, 0x19, 0x89, 0xfe, 0x46, 0xb7, 0xa1, 0xc0, 0x62, 0x21, 0x65, 0x6c, 0xa8, - 0x32, 0x35, 0xae, 0x19, 0x7a, 0x3b, 0xc7, 0xc8, 0x55, 0x43, 0x25, 0xa6, 0x14, 0x3d, 0x02, 0x4e, - 0x90, 0x0f, 0x3a, 0x36, 0xd1, 0x29, 0x90, 0x52, 0xe5, 0xc2, 0xd9, 0xe9, 0x62, 0xb6, 0x41, 0x6f, - 0x6c, 0x6c, 0x36, 0xea, 0x6b, 0x52, 0x96, 0x95, 0xda, 0xe8, 0xd8, 0x75, 0x6d, 0xe1, 0x37, 0x04, - 0x48, 0x3e, 0xad, 0x34, 0x2e, 0xac, 0x32, 0xb7, 0xa2, 0xc9, 0x40, 0x45, 0xef, 0x40, 0x61, 0x57, - 0x6f, 0xb7, 0x75, 0xa3, 0x45, 0xbc, 0xa0, 0xef, 0x62, 0xd5, 0x55, 0x58, 0x9e, 0x93, 0xb7, 0x19, - 0x15, 0x2d, 0x41, 0x56, 0xb5, 0xb0, 0x86, 0x0d, 0x47, 0x57, 0xda, 0x36, 0xd7, 0x5c, 0x90, 0xb4, - 0xf0, 0xcb, 0x02, 0x4c, 0xd2, 0x19, 0x80, 0xde, 0x86, 0x8c, 0x6a, 0x1a, 0x8e, 0xa2, 0x1b, 0xdc, - 0x94, 0x65, 0x24, 0x9f, 0x30, 0xb4, 0x7a, 0xd7, 0x61, 0x46, 0x51, 0x55, 0xb3, 0x67, 0x38, 0xb2, - 0xa1, 0x74, 0x30, 0xaf, 0x66, 0x96, 0xd3, 0xb6, 0x94, 0x0e, 0x46, 0x8b, 0xe0, 0x5e, 0x7a, 0x07, - 0x1d, 0x33, 0x12, 0x70, 0xd2, 0x06, 0x3e, 0x59, 0xf8, 0x63, 0x01, 0xd2, 0xee, 0x9c, 0x21, 0xd5, - 0x68, 0x61, 0x03, 0x5b, 0x8a, 0x63, 0x7a, 0xd5, 0xf0, 0x08, 0xfd, 0x4b, 0x65, 0xc6, 0x5f, 0x2a, - 0xe7, 0x61, 0xd2, 0x21, 0xd3, 0x82, 0xd7, 0x80, 0x5d, 0xd0, 0xed, 0xe8, 0xb6, 0xd2, 0x62, 0xbb, - 0x71, 0x19, 0x89, 0x5d, 0x90, 0xc6, 0xf0, 0x90, 0x5a, 0xa6, 0x11, 0x7e, 0x45, 0x6a, 0xca, 0x02, - 0x3f, 0x77, 0x71, 0x4b, 0x37, 0xe8, 0x58, 0x4a, 0x4a, 0x40, 0x49, 0x65, 0x42, 0x41, 0x6f, 0x41, - 0x86, 0x15, 0xc0, 0x86, 0x46, 0x07, 0x54, 0x52, 0x4a, 0x53, 0x42, 0xd5, 0xd0, 0x16, 0x30, 0x64, - 0xbc, 0xc9, 0x49, 0xba, 0xad, 0x67, 0x7b, 0x8a, 0xa4, 0xbf, 0xd1, 0xfb, 0x30, 0xff, 0xba, 0xa7, - 0xb4, 0xf5, 0x3d, 0xba, 0xd1, 0x46, 0x8a, 0x31, 0x9d, 0xb1, 0x96, 0x20, 0xef, 0x1e, 0x95, 0x40, - 0x55, 0xe7, 0xce, 0xe5, 0xa4, 0x3f, 0x97, 0xc5, 0xdf, 0x17, 0x60, 0x96, 0x45, 0xf5, 0xb0, 0x60, - 0xd4, 0xf8, 0xfc, 0x90, 0x8f, 0x21, 0xa3, 0x29, 0x8e, 0xc2, 0x8e, 0x6e, 0x26, 0x46, 0x1e, 0xdd, - 0xf4, 0x8e, 0x12, 0x28, 0x8e, 0x42, 0x8f, 0x6f, 0x22, 0x48, 0x91, 0xdf, 0xec, 0x94, 0xab, 0x44, - 0x7f, 0x8b, 0x9f, 0x01, 0x0a, 0x56, 0x34, 0x4e, 0x8f, 0xec, 0x1e, 0x5c, 0x22, 0xba, 0xae, 0x1a, - 0xaa, 0x75, 0xd2, 0x75, 0x74, 0xd3, 0x78, 0x4e, 0xff, 0xda, 0xa8, 0x18, 0x78, 0x2f, 0x45, 0x5f, - 0x47, 0x89, 0x7f, 0x38, 0x05, 0xb9, 0xea, 0x71, 0xd7, 0xb4, 0x62, 0xdd, 0xc5, 0x2a, 0xc3, 0x34, - 0x07, 0xfa, 0x23, 0xde, 0x0b, 0xf7, 0x19, 0x73, 0xf7, 0x95, 0x2b, 0x67, 0x44, 0x65, 0x00, 0x16, - 0x30, 0x4a, 0x83, 0x82, 0x92, 0x17, 0x78, 0x53, 0x46, 0xd9, 0x08, 0x15, 0x6d, 0x41, 0xb6, 0x73, - 0xa8, 0xaa, 0xf2, 0x9e, 0xde, 0x76, 0x78, 0xdc, 0x5d, 0x74, 0x88, 0xf8, 0xe6, 0xcb, 0x4a, 0xe5, - 0x09, 0x2d, 0xc4, 0x42, 0xe0, 0xfc, 0x6b, 0x09, 0x88, 0x04, 0xf6, 0x1b, 0xbd, 0x0b, 0xfc, 0x48, - 0x8d, 0x6c, 0xbb, 0xa7, 0xe7, 0xca, 0xb9, 0xb3, 0xd3, 0xc5, 0x8c, 0x44, 0xa9, 0x8d, 0x46, 0x53, - 0xca, 0xb0, 0x02, 0x0d, 0xdb, 0x41, 0x37, 0x20, 0x67, 0x76, 0x74, 0x47, 0x76, 0x9d, 0x24, 0xee, - 0x51, 0xce, 0x10, 0xa2, 0xeb, 0x44, 0xa1, 0x26, 0xdc, 0xc1, 0x06, 0x1d, 0xed, 0xa4, 0x9d, 0xf2, - 0x2e, 0xdb, 0x7c, 0x74, 0xd8, 0x8c, 0x96, 0xcd, 0xae, 0xa3, 0x77, 0xf4, 0x2f, 0xe8, 0x9b, 0x69, - 0xfe, 0xd2, 0xe8, 0x06, 0x2b, 0x4e, 0xda, 0x57, 0xa6, 0xbb, 0x92, 0xbc, 0xec, 0xf3, 0x40, 0x51, - 0xf4, 0xb7, 0x05, 0xb8, 0xcc, 0x15, 0x29, 0xef, 0xd2, 0x18, 0x77, 0xa5, 0xad, 0x3b, 0x27, 0xf2, - 0xc1, 0x61, 0x29, 0x4d, 0xfd, 0xd6, 0x9f, 0x8d, 0xec, 0x90, 0xc0, 0x38, 0x58, 0x76, 0xbb, 0xe5, - 0xe4, 0x19, 0x67, 0xde, 0x38, 0xac, 0x1a, 0x8e, 0x75, 0x52, 0xbe, 0x72, 0x76, 0xba, 0x38, 0x37, - 0x78, 0xf7, 0xa5, 0x34, 0x67, 0x0f, 0xb2, 0xa0, 0x1a, 0x00, 0xf6, 0xc6, 0x21, 0x5d, 0x31, 0xa2, - 0xfd, 0x8f, 0xc8, 0x01, 0x2b, 0x05, 0x78, 0xd1, 0x5d, 0x28, 0xf2, 0x93, 0x2d, 0x7b, 0x7a, 0x1b, - 0xcb, 0xb6, 0xfe, 0x05, 0xa6, 0x6b, 0x4b, 0x52, 0xca, 0x33, 0x3a, 0x11, 0xd1, 0xd0, 0xbf, 0xc0, - 0x0b, 0xdf, 0x85, 0xd2, 0xb0, 0xda, 0x07, 0xa7, 0x40, 0x86, 0xbd, 0x91, 0xfd, 0x28, 0xbc, 0x1d, - 0x33, 0xc6, 0x50, 0xe5, 0x5b, 0x32, 0x1f, 0x27, 0x3e, 0x12, 0xc4, 0x7f, 0x96, 0x80, 0x5c, 0xb9, - 0xd7, 0x3e, 0x78, 0xde, 0x6d, 0xf4, 0x3a, 0x1d, 0xc5, 0x3a, 0x21, 0x66, 0x90, 0x19, 0x0a, 0x52, - 0x41, 0x81, 0x99, 0x41, 0x6a, 0x09, 0xf4, 0x2f, 0x30, 0x59, 0x9c, 0x82, 0xa7, 0xb5, 0x59, 0x0c, - 0x3f, 0x6d, 0x43, 0xe0, 0x08, 0xb6, 0x79, 0x64, 0xa3, 0x8f, 0xa0, 0x14, 0x28, 0x48, 0xf7, 0x4e, - 0x64, 0x6c, 0x38, 0x96, 0x8e, 0xd9, 0xfe, 0x5f, 0x52, 0x0a, 0xc4, 0xcb, 0xd4, 0xc9, 0xed, 0x2a, - 0xbb, 0x8b, 0x9a, 0x30, 0x43, 0x0a, 0x9e, 0xc8, 0x74, 0x09, 0x71, 0xf7, 0x67, 0x1f, 0x44, 0x34, - 0x2b, 0x54, 0xef, 0x65, 0xaa, 0x9f, 0x0a, 0xe5, 0xa1, 0x3f, 0xa5, 0x2c, 0xf6, 0x29, 0x0b, 0x9f, - 0x42, 0xb1, 0xbf, 0x40, 0x50, 0x97, 0x29, 0xa6, 0xcb, 0xf9, 0xa0, 0x2e, 0x93, 0x01, 0x3d, 0xad, - 0xa7, 0xd2, 0xa9, 0xe2, 0xa4, 0xf8, 0x17, 0x49, 0xc8, 0xbb, 0xc3, 0x2c, 0x4e, 0xa0, 0x53, 0x86, - 0x49, 0x32, 0x28, 0xdc, 0x18, 0x8f, 0xdb, 0x23, 0x46, 0x37, 0x8f, 0x1a, 0x27, 0x83, 0xc5, 0x05, - 0xc9, 0x94, 0x35, 0x0e, 0x83, 0xb3, 0xf0, 0xcb, 0x09, 0x48, 0x51, 0x6c, 0xf1, 0x00, 0x52, 0x74, - 0xa1, 0x10, 0xc6, 0x59, 0x28, 0x68, 0x51, 0x6f, 0x39, 0x4b, 0x04, 0x5c, 0x53, 0xe2, 0xf3, 0xed, - 0x2b, 0x1f, 0x3c, 0x78, 0x48, 0x8d, 0xcd, 0x8c, 0xc4, 0xaf, 0x50, 0x99, 0x86, 0x1d, 0x99, 0x96, - 0x83, 0x35, 0xee, 0xd3, 0x2f, 0x9d, 0xd7, 0xbf, 0xee, 0xa2, 0xe4, 0xf2, 0xa1, 0xab, 0x90, 0x24, - 0x56, 0x6c, 0x9a, 0x05, 0x29, 0x9c, 0x9d, 0x2e, 0x26, 0x89, 0xfd, 0x22, 0x34, 0xb4, 0x02, 0xd9, - 0xb0, 0xc9, 0x20, 0x1e, 0x1c, 0x35, 0x8c, 0x81, 0xe9, 0x0e, 0x6d, 0x6f, 0x6a, 0x31, 0x3c, 0xcb, - 0xfb, 0xf8, 0x2f, 0x53, 0x90, 0xab, 0x77, 0xe2, 0x5e, 0x52, 0x56, 0xc3, 0x3d, 0x1c, 0x05, 0x84, - 0x42, 0x0f, 0x8d, 0xe8, 0xe0, 0xd0, 0x0a, 0x9e, 0xbc, 0xd8, 0x0a, 0x5e, 0x27, 0x9e, 0x32, 0x4f, - 0xbf, 0x90, 0x1c, 0x82, 0x79, 0xc2, 0xcf, 0xa7, 0x7e, 0x8a, 0x44, 0x78, 0xfc, 0x73, 0x14, 0x34, - 0xd0, 0xe4, 0x53, 0xea, 0x90, 0xb3, 0x51, 0x36, 0x35, 0xfe, 0x28, 0x9b, 0xc6, 0x86, 0x46, 0x17, - 0xb5, 0xb0, 0x45, 0x9d, 0x7e, 0x73, 0x8b, 0xba, 0xe0, 0xf0, 0xc1, 0xfa, 0x31, 0x24, 0x35, 0xdd, - 0xed, 0x9c, 0xf1, 0x97, 0x6a, 0xc2, 0x74, 0xce, 0xa8, 0x4d, 0x05, 0x47, 0x2d, 0x1b, 0x25, 0x0b, - 0x75, 0x00, 0x5f, 0x37, 0x68, 0x09, 0xa6, 0xcc, 0xb6, 0xe6, 0x1e, 0x24, 0xc9, 0x95, 0x33, 0x67, - 0xa7, 0x8b, 0x93, 0xcf, 0xdb, 0x5a, 0x7d, 0x4d, 0x9a, 0x34, 0xdb, 0x5a, 0x5d, 0xa3, 0xb9, 0x2f, - 0xf0, 0x91, 0xec, 0x45, 0x99, 0xcd, 0x48, 0xd3, 0x06, 0x3e, 0x5a, 0xc3, 0xb6, 0xca, 0x07, 0xdc, - 0x6f, 0x0b, 0x90, 0x77, 0x75, 0x1f, 0xaf, 0x51, 0x49, 0xeb, 0x1d, 0x3e, 0xc9, 0x92, 0x17, 0x9b, - 0x64, 0x2e, 0x1f, 0x3f, 0x5e, 0xfb, 0xab, 0x02, 0x8f, 0x1b, 0x6e, 0xa8, 0x8a, 0x43, 0x9c, 0x8a, - 0x18, 0x27, 0xc6, 0x3d, 0x28, 0x5a, 0x8a, 0xa1, 0x99, 0x1d, 0xfd, 0x0b, 0xcc, 0x36, 0x42, 0x6d, - 0xfe, 0xd6, 0xb2, 0xe0, 0xd1, 0xe9, 0xae, 0x9f, 0x2d, 0xfe, 0x71, 0x82, 0xc7, 0x18, 0x7b, 0xd5, - 0x88, 0x53, 0x5d, 0xdf, 0x81, 0xd9, 0xfe, 0x44, 0x24, 0xee, 0x6c, 0x7d, 0x2f, 0x42, 0x5e, 0x54, - 0x45, 0x58, 0xac, 0xa0, 0x1b, 0xb8, 0xde, 0x97, 0x94, 0xc4, 0x46, 0x15, 0xc8, 0x06, 0xf3, 0x9b, - 0x24, 0xc7, 0xce, 0x6f, 0x02, 0x96, 0x97, 0xd5, 0x64, 0xe1, 0xe7, 0x61, 0x92, 0xde, 0x7e, 0x03, - 0x13, 0xcd, 0x7b, 0xf3, 0xcf, 0x13, 0x70, 0x93, 0xd6, 0xfe, 0x25, 0xb6, 0xf4, 0xbd, 0x93, 0x6d, - 0xcb, 0x74, 0xb0, 0xea, 0x60, 0xcd, 0x3f, 0x05, 0x12, 0xab, 0xdd, 0xcb, 0x74, 0xdd, 0x07, 0x5c, - 0x28, 0x5e, 0xcc, 0xe3, 0x42, 0x1b, 0x50, 0xe0, 0x91, 0x01, 0x4a, 0x5b, 0x3f, 0xc4, 0xb2, 0xe2, - 0x5c, 0x64, 0x75, 0xcb, 0x31, 0xde, 0x55, 0xc2, 0xba, 0xea, 0x20, 0x0d, 0x32, 0x5c, 0x98, 0xae, - 0xf1, 0xb4, 0x3c, 0x4f, 0x7f, 0xb2, 0x0d, 0xc5, 0x34, 0x0b, 0x4f, 0xa8, 0xaf, 0x49, 0x69, 0x26, - 0xb9, 0xae, 0x89, 0xff, 0x45, 0x80, 0x5b, 0xe7, 0xa8, 0x38, 0xce, 0xa1, 0xbb, 0x00, 0xe9, 0x43, - 0xf2, 0x20, 0x9d, 0xeb, 0x38, 0x2d, 0x79, 0xd7, 0x68, 0x13, 0x72, 0x7b, 0x8a, 0xde, 0xf6, 0x87, - 0xf4, 0xf0, 0xf0, 0xc2, 0xe8, 0x48, 0xd7, 0x19, 0xc6, 0xce, 0xc6, 0xb0, 0xf8, 0x9b, 0x09, 0x98, - 0x5d, 0xd5, 0xb4, 0x46, 0x83, 0xdb, 0xc0, 0xf8, 0x46, 0x8a, 0x0b, 0x32, 0x13, 0x3e, 0xc8, 0x44, - 0xef, 0x01, 0xd2, 0x74, 0x9b, 0xa5, 0xff, 0xb0, 0xf7, 0x15, 0xcd, 0x3c, 0xf2, 0xc3, 0x2a, 0x66, - 0xdd, 0x3b, 0x0d, 0xf7, 0x06, 0x6a, 0x00, 0x45, 0x3b, 0xb2, 0xed, 0x28, 0xde, 0x7b, 0xa3, 0x5b, - 0x63, 0x1d, 0xd7, 0x62, 0x30, 0xc8, 0xbb, 0x94, 0x32, 0x44, 0x0e, 0xfd, 0x49, 0xfc, 0x76, 0x9d, - 0x34, 0xdd, 0x91, 0x15, 0xdb, 0x3d, 0x9b, 0xc3, 0x12, 0x8f, 0xe4, 0x19, 0x7d, 0xd5, 0x66, 0x47, - 0x6e, 0xd8, 0x61, 0x02, 0x5f, 0x35, 0x71, 0x42, 0xe2, 0x7f, 0x24, 0x40, 0x5e, 0xc2, 0x7b, 0x16, - 0xb6, 0x63, 0xdd, 0x14, 0x78, 0x02, 0x33, 0x16, 0x93, 0x2a, 0xef, 0x59, 0x66, 0xe7, 0x22, 0xf3, - 0x2a, 0xcb, 0x19, 0x9f, 0x58, 0x66, 0x87, 0x1b, 0x96, 0x97, 0x50, 0xf0, 0xea, 0x18, 0x67, 0xe3, - 0x7f, 0x9f, 0x1e, 0x45, 0x66, 0x82, 0xe3, 0x8e, 0x6f, 0x88, 0x57, 0x03, 0xf4, 0x45, 0x55, 0xb0, - 0xa2, 0x71, 0xaa, 0xe1, 0x7f, 0x08, 0x90, 0x6f, 0xf4, 0x76, 0x59, 0x7e, 0xa9, 0xf8, 0x34, 0x50, - 0x85, 0x4c, 0x1b, 0xef, 0x39, 0xf2, 0x1b, 0x85, 0xc1, 0xa7, 0x09, 0x2b, 0x3d, 0x04, 0xf0, 0x14, - 0xc0, 0xa2, 0xc7, 0xd7, 0xa8, 0x9c, 0xe4, 0x05, 0xe5, 0x64, 0x28, 0x2f, 0x21, 0x8b, 0xff, 0x3c, - 0x01, 0x05, 0xaf, 0x99, 0x71, 0x5a, 0xc9, 0x57, 0x21, 0xeb, 0x90, 0xbc, 0x88, 0x75, 0x98, 0xe5, - 0x21, 0x1d, 0xd1, 0x16, 0x62, 0x19, 0xe6, 0xa8, 0xe3, 0x22, 0x2b, 0xdd, 0x6e, 0x5b, 0x77, 0xe1, - 0x2e, 0xb5, 0x3f, 0x29, 0x69, 0x96, 0xde, 0x5a, 0x65, 0x77, 0x28, 0xd0, 0x25, 0x63, 0x6e, 0xcf, - 0xc2, 0xf8, 0x0b, 0x2c, 0x53, 0xe4, 0x75, 0x91, 0x90, 0x95, 0x2c, 0x63, 0x6c, 0x10, 0x3e, 0x3e, - 0xe6, 0x5e, 0xc1, 0x2c, 0xd5, 0x69, 0xdc, 0xc7, 0x6e, 0xc5, 0x7f, 0x90, 0x00, 0x14, 0x94, 0xfc, - 0xd5, 0xf5, 0x45, 0x22, 0xbe, 0xbe, 0x78, 0x17, 0x10, 0x0b, 0x5a, 0xb4, 0xe5, 0x2e, 0xb6, 0x64, - 0x1b, 0xab, 0x26, 0xcf, 0x71, 0x24, 0x48, 0x45, 0x7e, 0x67, 0x1b, 0x5b, 0x0d, 0x4a, 0x47, 0x8f, - 0x01, 0x7c, 0x8f, 0x8c, 0x2f, 0x18, 0x23, 0x1d, 0x32, 0x29, 0xe3, 0xb9, 0x62, 0xe2, 0xf7, 0x16, - 0x60, 0x86, 0xeb, 0x70, 0xc7, 0xd0, 0x4d, 0x03, 0x3d, 0x80, 0x64, 0x8b, 0x6f, 0xfd, 0x67, 0x23, - 0xb7, 0xe9, 0xfc, 0xe4, 0x6e, 0xb5, 0x09, 0x89, 0x94, 0x25, 0x2c, 0xdd, 0x9e, 0x13, 0xe1, 0x18, - 0xf9, 0x81, 0xd8, 0x41, 0x96, 0x6e, 0xcf, 0x41, 0x0d, 0x28, 0xa8, 0x7e, 0x92, 0x2a, 0x99, 0xb0, - 0x27, 0x87, 0x82, 0xa8, 0xc8, 0x74, 0x61, 0xb5, 0x09, 0x29, 0xaf, 0x86, 0x6e, 0xa0, 0x4a, 0x30, - 0x37, 0x52, 0x6a, 0x20, 0xca, 0xcb, 0x3f, 0xdf, 0x1b, 0xce, 0xcb, 0x54, 0x9b, 0x08, 0xa4, 0x50, - 0x42, 0x1f, 0xc3, 0x94, 0x46, 0xb3, 0xf0, 0xf0, 0x11, 0x1d, 0x35, 0xe8, 0x42, 0xc9, 0x8e, 0x6a, - 0x13, 0x12, 0xe7, 0x40, 0xeb, 0x30, 0xc3, 0x7e, 0x31, 0x37, 0x85, 0x23, 0xcb, 0x5b, 0xc3, 0x25, - 0x04, 0x16, 0x82, 0xda, 0x84, 0x94, 0xd5, 0x7c, 0x2a, 0xfa, 0x26, 0xa4, 0x6c, 0x55, 0x71, 0xb1, - 0xe5, 0xb5, 0x21, 0x29, 0x35, 0x7c, 0x66, 0x5a, 0x1a, 0x3d, 0x66, 0xe9, 0x1a, 0x9d, 0x63, 0x77, - 0x9b, 0x2f, 0xaa, 0xfa, 0xa1, 0xa3, 0xdb, 0xa4, 0xfa, 0x98, 0x12, 0xd0, 0x53, 0xc8, 0x2a, 0xc4, - 0xdf, 0x93, 0xe9, 0xe1, 0x47, 0xba, 0xaf, 0x17, 0xfd, 0x72, 0x7d, 0xe0, 0xb0, 0x6a, 0x8d, 0x9e, - 0x0f, 0x77, 0x89, 0xbe, 0xa0, 0x0e, 0xb6, 0x5a, 0xb8, 0x94, 0x1d, 0x2d, 0x28, 0x18, 0xdb, 0xe5, - 0x09, 0xa2, 0x44, 0xe2, 0xf7, 0x79, 0x27, 0x8f, 0x69, 0xa3, 0x66, 0x86, 0xbe, 0xce, 0x8d, 0x38, - 0xb6, 0x53, 0x9b, 0x90, 0x66, 0xf6, 0x03, 0x64, 0xb4, 0x0c, 0x89, 0x96, 0x5a, 0xca, 0x0d, 0x9d, - 0x21, 0xde, 0xd1, 0x94, 0xda, 0x84, 0x94, 0x68, 0xa9, 0xe8, 0x53, 0x48, 0xb3, 0x73, 0x06, 0xc7, - 0x46, 0x29, 0x3f, 0xd4, 0x4e, 0x84, 0x4f, 0x6b, 0xd4, 0x26, 0x24, 0x7a, 0xb4, 0x81, 0x3c, 0x6f, - 0x1b, 0xf2, 0x16, 0x0b, 0x8e, 0x73, 0x43, 0x59, 0x8b, 0x43, 0x5f, 0x71, 0x47, 0x45, 0xb3, 0xd6, - 0xa8, 0xe7, 0x1f, 0xa0, 0xa3, 0xef, 0xc0, 0x7c, 0x58, 0x22, 0x1f, 0x69, 0xb3, 0x43, 0x5f, 0xd7, - 0x0e, 0x8d, 0xac, 0xac, 0x4d, 0x48, 0xc8, 0x1a, 0xb8, 0x89, 0x3e, 0x84, 0x49, 0xd6, 0x6b, 0x88, - 0x8a, 0x8c, 0x8a, 0xda, 0xe8, 0xeb, 0x30, 0x56, 0x9e, 0x0c, 0x7e, 0x87, 0x47, 0x88, 0xc9, 0x6d, - 0xb3, 0x55, 0x9a, 0x1b, 0x3a, 0xf8, 0x07, 0x63, 0xdd, 0xc8, 0xe0, 0x77, 0x7c, 0x2a, 0xe9, 0x77, - 0x8b, 0xdd, 0xe1, 0x61, 0x45, 0xf3, 0x43, 0xfb, 0x3d, 0x22, 0x70, 0xac, 0x46, 0x63, 0xf7, 0x7d, - 0x32, 0xa9, 0x9a, 0xc5, 0xb2, 0xc0, 0xc8, 0x74, 0x4e, 0x5d, 0x1a, 0x5a, 0xb5, 0xc1, 0x34, 0x39, - 0x35, 0xea, 0x23, 0x79, 0x54, 0xf4, 0x12, 0x8a, 0x3c, 0x57, 0x83, 0xff, 0x4e, 0xe1, 0x32, 0x95, - 0x77, 0x2f, 0xd2, 0x74, 0x45, 0xc5, 0xe4, 0xd4, 0x26, 0xa4, 0x82, 0x1a, 0xbe, 0x83, 0x3e, 0x83, - 0x59, 0x2a, 0x4f, 0x56, 0xfd, 0xf4, 0x1a, 0xa5, 0xd2, 0x40, 0xb2, 0x86, 0xe1, 0x99, 0x38, 0x5c, - 0xc9, 0x45, 0xb5, 0xef, 0x16, 0x19, 0xc6, 0xba, 0xa1, 0x3b, 0xd4, 0xca, 0x2e, 0x0c, 0x1d, 0xc6, - 0xe1, 0xa4, 0x7e, 0x64, 0x18, 0xeb, 0x8c, 0x42, 0x86, 0xb1, 0xc3, 0x63, 0xce, 0x78, 0x77, 0xbc, - 0x3d, 0x74, 0x18, 0x47, 0x05, 0xa7, 0x91, 0x61, 0xec, 0x04, 0xe9, 0x64, 0x18, 0x33, 0x03, 0xd1, - 0x27, 0xf7, 0x9d, 0xa1, 0xc3, 0x78, 0xe8, 0x21, 0x64, 0x32, 0x8c, 0x95, 0x81, 0x9b, 0x68, 0x0d, - 0x80, 0xb9, 0x33, 0x74, 0x51, 0xbc, 0x36, 0x74, 0x31, 0xe8, 0x8f, 0x3d, 0x23, 0x8b, 0x41, 0xdb, - 0xa5, 0x11, 0x43, 0x46, 0xc1, 0x92, 0x4c, 0x5f, 0xa1, 0x96, 0x16, 0x87, 0x1a, 0xb2, 0x81, 0xd7, - 0x9d, 0xc4, 0x90, 0x1d, 0x79, 0x44, 0xb2, 0xaa, 0xb0, 0x3d, 0xdf, 0xd2, 0xd2, 0x70, 0xb3, 0x1c, - 0x7c, 0xf5, 0x43, 0xcd, 0x32, 0x25, 0xa0, 0x55, 0xc8, 0x90, 0x35, 0xff, 0x84, 0x9a, 0xa1, 0xeb, - 0x43, 0x7d, 0xd2, 0xbe, 0xc3, 0x29, 0xb5, 0x09, 0x29, 0xfd, 0x9a, 0x93, 0xc8, 0xe3, 0xd9, 0x6e, - 0x58, 0x49, 0x1c, 0xfa, 0xf8, 0xd0, 0xce, 0x29, 0x79, 0x3c, 0xe3, 0x40, 0x2a, 0x5c, 0x62, 0x7d, - 0xc5, 0xcf, 0x03, 0x5b, 0xfc, 0x08, 0x6b, 0xe9, 0x06, 0x15, 0x35, 0x74, 0x5b, 0x29, 0xf2, 0x98, - 0x72, 0x6d, 0x42, 0x9a, 0x53, 0x06, 0xef, 0x92, 0x09, 0xcf, 0x97, 0x1e, 0xb6, 0x19, 0x55, 0xba, - 0x39, 0x74, 0xc2, 0x47, 0xec, 0xe1, 0x91, 0x09, 0xaf, 0x04, 0xc8, 0x6c, 0x01, 0xd2, 0x64, 0xdb, - 0x66, 0x2f, 0xdc, 0x6f, 0x8d, 0x58, 0x80, 0xfa, 0x76, 0x01, 0xd8, 0x02, 0xa4, 0x35, 0x18, 0x27, - 0x11, 0xa4, 0xb6, 0xb1, 0x62, 0x71, 0x33, 0x7b, 0x7b, 0xa8, 0xa0, 0x81, 0xfc, 0x79, 0x44, 0x90, - 0xea, 0x11, 0x89, 0xc3, 0x63, 0xb9, 0x49, 0x5e, 0xb8, 0xc3, 0x78, 0x67, 0xa8, 0xc3, 0x13, 0x99, - 0x85, 0x86, 0x38, 0x3c, 0x56, 0xe8, 0x06, 0xfa, 0x39, 0x98, 0xe6, 0xf0, 0xad, 0x74, 0x77, 0x84, - 0x1b, 0x1b, 0x44, 0xdc, 0x64, 0x5e, 0x73, 0x1e, 0x66, 0x65, 0x19, 0x6c, 0x64, 0xcd, 0xbb, 0x37, - 0xc2, 0xca, 0x0e, 0x20, 0x57, 0x66, 0x65, 0x7d, 0x32, 0xb1, 0xb2, 0x6c, 0x9c, 0xf2, 0xb5, 0xee, - 0xfe, 0x50, 0x2b, 0x3b, 0x78, 0xf4, 0x85, 0x58, 0xd9, 0xd7, 0x3e, 0x95, 0xb4, 0xcc, 0x66, 0xf0, - 0xa9, 0xf4, 0x8d, 0xa1, 0x2d, 0x0b, 0xe3, 0x48, 0xd2, 0x32, 0xce, 0x43, 0xba, 0x8d, 0xb9, 0xc4, - 0x4c, 0xd3, 0xef, 0x0e, 0x3f, 0x74, 0xdf, 0x0f, 0x3a, 0x6a, 0xee, 0x46, 0x25, 0xd3, 0xb0, 0x67, - 0xa8, 0x2c, 0x7e, 0xe8, 0x98, 0x6b, 0xea, 0xbd, 0xd1, 0x86, 0x2a, 0xea, 0x24, 0xb5, 0x67, 0xa8, - 0x42, 0x37, 0x69, 0x55, 0xd9, 0x39, 0x33, 0x3a, 0xbf, 0x97, 0x47, 0xe4, 0x07, 0xe8, 0x3b, 0xed, - 0x47, 0xab, 0xea, 0x11, 0xfd, 0x29, 0xd4, 0x63, 0xc9, 0x2b, 0x4a, 0x2b, 0xa3, 0xa7, 0x50, 0x38, - 0x7d, 0x86, 0x37, 0x85, 0x38, 0xd9, 0x5b, 0x33, 0x5d, 0x0f, 0xe3, 0xfd, 0xd1, 0x6b, 0x66, 0xbf, - 0x6b, 0xc1, 0xd6, 0x4c, 0xee, 0x53, 0xfc, 0x4d, 0x01, 0x96, 0x58, 0xdd, 0xe8, 0x8e, 0xde, 0x89, - 0xec, 0xed, 0x8b, 0x06, 0xce, 0x3c, 0x3c, 0xa0, 0x0f, 0xf8, 0x70, 0x58, 0x75, 0xcf, 0xd9, 0xe7, - 0xad, 0x4d, 0x48, 0xef, 0x28, 0xa3, 0xca, 0x95, 0xa7, 0xf9, 0x6b, 0x51, 0xef, 0x90, 0x67, 0xa1, - 0x58, 0x5c, 0x4f, 0xa5, 0xaf, 0x14, 0x4b, 0xeb, 0xa9, 0xf4, 0xd5, 0xe2, 0xc2, 0x7a, 0x2a, 0xfd, - 0x56, 0xf1, 0x6d, 0xf1, 0x2f, 0xaf, 0x42, 0xce, 0x45, 0x7e, 0x0c, 0x11, 0x3d, 0x0c, 0x22, 0xa2, - 0x6b, 0xc3, 0x10, 0x11, 0xc7, 0x8a, 0x1c, 0x12, 0x3d, 0x0c, 0x42, 0xa2, 0x6b, 0xc3, 0x20, 0x91, - 0xcf, 0x43, 0x30, 0x51, 0x73, 0x18, 0x26, 0xba, 0x37, 0x06, 0x26, 0xf2, 0x44, 0xf5, 0x83, 0xa2, - 0xb5, 0x41, 0x50, 0x74, 0x73, 0x34, 0x28, 0xf2, 0x44, 0x05, 0x50, 0xd1, 0xe3, 0x3e, 0x54, 0x74, - 0x7d, 0x04, 0x2a, 0xf2, 0xf8, 0x5d, 0x58, 0xb4, 0x11, 0x09, 0x8b, 0x6e, 0x9f, 0x07, 0x8b, 0x3c, - 0x39, 0x21, 0x5c, 0xf4, 0x41, 0x08, 0x17, 0x2d, 0x0e, 0xc5, 0x45, 0x1e, 0x37, 0x03, 0x46, 0x9f, - 0xf4, 0x03, 0xa3, 0xeb, 0x23, 0x80, 0x91, 0xdf, 0x02, 0x8e, 0x8c, 0x6a, 0x51, 0xc8, 0xe8, 0xd6, - 0x39, 0xc8, 0xc8, 0x93, 0x12, 0x84, 0x46, 0xb5, 0x28, 0x68, 0x74, 0xeb, 0x1c, 0x68, 0xd4, 0x27, - 0x89, 0x61, 0xa3, 0xad, 0x68, 0x6c, 0x74, 0xe7, 0x5c, 0x6c, 0xe4, 0x49, 0x0b, 0x83, 0xa3, 0x95, - 0x00, 0x38, 0x7a, 0x67, 0x08, 0x38, 0xf2, 0x58, 0x09, 0x3a, 0xfa, 0xd6, 0x00, 0x3a, 0x12, 0x47, - 0xa1, 0x23, 0x8f, 0xd7, 0x83, 0x47, 0x2f, 0x86, 0xc0, 0xa3, 0xbb, 0xe7, 0xc3, 0x23, 0x4f, 0x58, - 0x1f, 0x3e, 0x52, 0x46, 0xe2, 0xa3, 0xf7, 0xc6, 0xc4, 0x47, 0x9e, 0xf4, 0x28, 0x80, 0xf4, 0x51, - 0x18, 0x20, 0x2d, 0x0d, 0x07, 0x48, 0x9e, 0x18, 0x8e, 0x90, 0x36, 0x22, 0x11, 0xd2, 0xed, 0xf3, - 0x10, 0x92, 0x3f, 0x0f, 0x82, 0x10, 0x69, 0x2b, 0x1a, 0x22, 0xdd, 0x39, 0x17, 0x22, 0xf9, 0xdd, - 0x1f, 0xc2, 0x48, 0x1b, 0x91, 0x18, 0xe9, 0xf6, 0x79, 0x18, 0xc9, 0xaf, 0x5c, 0x10, 0x24, 0xbd, - 0x1a, 0x0a, 0x92, 0xee, 0x8f, 0x03, 0x92, 0x3c, 0xa1, 0x03, 0x28, 0xe9, 0xf3, 0xe1, 0x28, 0xe9, - 0x1b, 0x17, 0xc8, 0x57, 0x18, 0x09, 0x93, 0xbe, 0x35, 0x00, 0x93, 0xc4, 0x51, 0x30, 0xc9, 0x1f, - 0xcf, 0x2e, 0x4e, 0x52, 0x46, 0xa2, 0x9a, 0xf7, 0xc6, 0x44, 0x35, 0xfe, 0xe0, 0x8b, 0x80, 0x35, - 0xd5, 0x08, 0x58, 0x73, 0x73, 0x34, 0xac, 0xf1, 0xcd, 0xb9, 0x8f, 0x6b, 0x6a, 0x51, 0xb8, 0xe6, - 0xd6, 0x39, 0xb8, 0xc6, 0xb7, 0x42, 0x01, 0x60, 0xf3, 0xb8, 0x0f, 0xd8, 0x5c, 0x3f, 0x37, 0xea, - 0x27, 0x80, 0x6c, 0xca, 0x83, 0xc8, 0xe6, 0xc6, 0x48, 0x64, 0xe3, 0x49, 0xf0, 0xa1, 0xcd, 0xe3, - 0x3e, 0x68, 0x73, 0x7d, 0x04, 0xb4, 0xf1, 0x2b, 0xc0, 0xb1, 0x8d, 0x36, 0x1a, 0xdb, 0x2c, 0x8f, - 0x8b, 0x6d, 0x3c, 0xc1, 0x91, 0xe0, 0x66, 0x2b, 0x1a, 0xdc, 0xdc, 0x19, 0xf3, 0x85, 0xfc, 0x00, - 0xba, 0xa9, 0x45, 0xa1, 0x9b, 0x5b, 0xe7, 0xa0, 0x9b, 0xe0, 0x1a, 0xe2, 0xc1, 0x9b, 0x5a, 0x14, - 0xbc, 0xb9, 0x75, 0x0e, 0xbc, 0xf1, 0x25, 0x05, 0xf0, 0x4d, 0x73, 0x18, 0xbe, 0xb9, 0x37, 0x06, - 0xbe, 0xf1, 0x9d, 0x97, 0x3e, 0x80, 0xf3, 0x69, 0x3f, 0xc0, 0x11, 0x47, 0x01, 0x1c, 0x7f, 0x46, - 0xba, 0x08, 0x67, 0x2b, 0x1a, 0xe1, 0xdc, 0x39, 0x17, 0xe1, 0x04, 0x8d, 0x64, 0x00, 0xe2, 0x6c, - 0x44, 0x42, 0x9c, 0xdb, 0xe7, 0x41, 0x1c, 0xdf, 0x48, 0x06, 0x31, 0xce, 0xa7, 0xfd, 0x18, 0x47, - 0x1c, 0x85, 0x71, 0xfc, 0xc6, 0xb9, 0x20, 0xa7, 0x16, 0x05, 0x72, 0x6e, 0x9d, 0x03, 0x72, 0xfc, - 0xce, 0x0b, 0xa0, 0x1c, 0x65, 0x24, 0xca, 0x79, 0x6f, 0x4c, 0x94, 0xd3, 0x67, 0xb8, 0xc2, 0x30, - 0xa7, 0x16, 0x05, 0x73, 0x6e, 0x9d, 0x03, 0x73, 0x02, 0x95, 0xf5, 0x71, 0xce, 0x56, 0x34, 0xce, - 0xb9, 0x73, 0x2e, 0xce, 0xe9, 0x9b, 0x4d, 0x2e, 0xd0, 0xd9, 0x88, 0x04, 0x3a, 0xb7, 0xcf, 0x03, - 0x3a, 0x7d, 0x0b, 0x1f, 0x77, 0x0e, 0x7e, 0x65, 0x7c, 0xa4, 0xf3, 0xd1, 0xc5, 0x91, 0x8e, 0xf7, - 0xcc, 0x58, 0xa0, 0xce, 0x7a, 0x2a, 0xfd, 0x76, 0xf1, 0x1d, 0xf1, 0xef, 0x4c, 0xc3, 0x54, 0xcd, - 0x8b, 0x73, 0xf1, 0x6b, 0x29, 0xbc, 0x49, 0x5e, 0x24, 0xb4, 0x46, 0x66, 0x2c, 0xb5, 0x7b, 0xe7, - 0xa7, 0xba, 0x1b, 0x4c, 0xc6, 0xc6, 0x59, 0xdf, 0xe0, 0x80, 0x32, 0xfa, 0x00, 0x72, 0x3d, 0x1b, - 0x5b, 0x72, 0xd7, 0xd2, 0x4d, 0x4b, 0x77, 0xd8, 0x29, 0x0e, 0xa1, 0x5c, 0xfc, 0xf2, 0x74, 0x71, - 0x66, 0xc7, 0xc6, 0xd6, 0x36, 0xa7, 0x4b, 0x33, 0xbd, 0xc0, 0x95, 0xfb, 0x01, 0xa9, 0xc9, 0xf1, - 0x3f, 0x20, 0xf5, 0x02, 0x8a, 0x16, 0x56, 0xb4, 0x90, 0x07, 0xc2, 0x72, 0x10, 0x45, 0x8f, 0x19, - 0x7a, 0xca, 0xca, 0x2d, 0x49, 0x73, 0x11, 0x15, 0xac, 0x30, 0x11, 0x3d, 0x80, 0x4b, 0x1d, 0xe5, - 0x98, 0xc6, 0x44, 0xca, 0xae, 0x53, 0x47, 0xe3, 0x1c, 0xd9, 0xb7, 0x99, 0x50, 0x47, 0x39, 0xa6, - 0x5f, 0xa3, 0x62, 0xb7, 0xe8, 0xa7, 0x24, 0x6e, 0x41, 0x5e, 0xd3, 0x6d, 0x47, 0x37, 0x54, 0x87, - 0xe7, 0x9e, 0x65, 0x79, 0x5b, 0x73, 0x2e, 0x95, 0x25, 0x98, 0xbd, 0x0f, 0xb3, 0x3c, 0x58, 0x3e, - 0xf0, 0x8a, 0x90, 0xe7, 0x6f, 0x65, 0x37, 0xbc, 0xb7, 0x82, 0xa8, 0x02, 0x85, 0x96, 0xe2, 0xe0, - 0x23, 0xe5, 0x44, 0x76, 0x8f, 0x62, 0x65, 0x69, 0x4a, 0xc6, 0xb7, 0xce, 0x4e, 0x17, 0x73, 0x4f, - 0xd9, 0xad, 0x81, 0x13, 0x59, 0xb9, 0x56, 0xe0, 0x86, 0x86, 0xee, 0x40, 0x41, 0xb1, 0x4f, 0x0c, - 0x95, 0xaa, 0x07, 0x1b, 0x76, 0xcf, 0xa6, 0x90, 0x22, 0x2d, 0xe5, 0x29, 0xb9, 0xe2, 0x52, 0xd1, - 0x75, 0x98, 0xe1, 0x91, 0xe4, 0xec, 0xf3, 0x36, 0x05, 0xda, 0x54, 0xfe, 0xdd, 0x04, 0xf6, 0x85, - 0x9b, 0xc7, 0xb0, 0xc0, 0x73, 0xcb, 0x1f, 0x29, 0x96, 0x26, 0x53, 0xad, 0xfb, 0xe3, 0xb3, 0x48, - 0xc5, 0x5e, 0x61, 0xb9, 0xe4, 0x49, 0x01, 0xa2, 0x6a, 0x3f, 0x13, 0xc2, 0x16, 0xcc, 0xaa, 0x6d, - 0xdd, 0x43, 0x00, 0xac, 0xe5, 0xb3, 0x43, 0xed, 0x6c, 0x85, 0x96, 0xf5, 0x5f, 0x91, 0x16, 0xd4, - 0x30, 0x01, 0x35, 0x80, 0x66, 0x7b, 0x91, 0xbb, 0x66, 0x5b, 0x57, 0x4f, 0xa8, 0xf3, 0x1f, 0xce, - 0x91, 0x3d, 0x32, 0x53, 0xfd, 0x2b, 0x45, 0x77, 0xb6, 0x29, 0xa7, 0x04, 0x47, 0xde, 0x6f, 0x96, - 0xdf, 0x76, 0x3d, 0x95, 0x9e, 0x29, 0xe6, 0xd6, 0x53, 0xe9, 0x7c, 0xb1, 0x20, 0xfe, 0x5d, 0x01, - 0x0a, 0x7d, 0x75, 0x41, 0x35, 0xb8, 0xa4, 0x79, 0x53, 0x45, 0xe6, 0x47, 0x8d, 0x74, 0xd3, 0xe0, - 0x69, 0xbf, 0xe7, 0xbe, 0x3c, 0x5d, 0x2c, 0xd0, 0xd2, 0x4f, 0xbd, 0x5b, 0xd2, 0xbc, 0xcf, 0xe1, - 0x53, 0xd1, 0x47, 0x90, 0x67, 0xee, 0xa3, 0xf7, 0xc5, 0x36, 0x1a, 0x23, 0x5e, 0x9e, 0xfd, 0xf2, - 0x74, 0x31, 0x47, 0x7d, 0x46, 0x37, 0x2d, 0xaf, 0x94, 0x6b, 0x07, 0x2f, 0xc5, 0xdf, 0x10, 0x60, - 0x26, 0x74, 0x9c, 0xe7, 0x71, 0xdf, 0x1b, 0xf4, 0xab, 0xd1, 0xb8, 0x73, 0x58, 0x40, 0x5d, 0x9a, - 0x8f, 0x73, 0x37, 0x3a, 0x71, 0x71, 0x38, 0x6e, 0xa1, 0xbb, 0x30, 0x6e, 0xa8, 0x86, 0xcb, 0xf6, - 0x71, 0xea, 0xfb, 0x3f, 0x58, 0x9c, 0x10, 0xff, 0x20, 0x05, 0xb9, 0xf0, 0xe1, 0x9d, 0x7a, 0x5f, - 0xbd, 0xa2, 0xd6, 0x85, 0x10, 0xc7, 0xf2, 0x88, 0xb4, 0x85, 0x19, 0x3f, 0xbf, 0x3e, 0xab, 0xe6, - 0xd2, 0x88, 0x38, 0x81, 0x60, 0x3d, 0x7d, 0xc6, 0x85, 0xef, 0x25, 0x3d, 0xfb, 0xba, 0x0c, 0x93, - 0x34, 0x8d, 0x0e, 0xaf, 0x5a, 0xa9, 0x7f, 0xf4, 0x10, 0x5f, 0x99, 0xdc, 0x97, 0x58, 0x31, 0x62, - 0x8f, 0x9b, 0x6f, 0x94, 0xa7, 0xce, 0x9f, 0x06, 0x17, 0xff, 0x40, 0x1e, 0xcf, 0x53, 0x38, 0x79, - 0xb1, 0x3c, 0x85, 0xe8, 0x97, 0xa0, 0xa0, 0x9a, 0xed, 0x36, 0x5b, 0xeb, 0x98, 0x45, 0x1a, 0xcc, - 0x3c, 0x42, 0x45, 0xf0, 0x6f, 0x1a, 0x2e, 0x7b, 0xdf, 0x36, 0x5c, 0x96, 0xf8, 0xb7, 0x0d, 0x03, - 0xb1, 0x9e, 0x79, 0x4f, 0x18, 0x33, 0x64, 0x7d, 0x61, 0xa7, 0xd3, 0x6f, 0x12, 0x76, 0xca, 0x42, - 0x95, 0xf9, 0xc8, 0xf9, 0x53, 0x81, 0x07, 0x86, 0x3c, 0x33, 0xcd, 0x83, 0x9e, 0x17, 0x2e, 0xba, - 0x10, 0xcc, 0x1a, 0x98, 0xfe, 0xf2, 0x74, 0x31, 0x25, 0x79, 0x69, 0x03, 0xa3, 0x2c, 0x7f, 0xe2, - 0x27, 0xb3, 0xfc, 0xd7, 0x61, 0xa6, 0x6b, 0xe1, 0x3d, 0xec, 0xa8, 0xfb, 0xb2, 0xd1, 0xeb, 0xf0, - 0x53, 0x25, 0x59, 0x97, 0xb6, 0xd5, 0xeb, 0xa0, 0x7b, 0x50, 0xf4, 0x8a, 0x70, 0x8c, 0xed, 0xa6, - 0x95, 0x72, 0xe9, 0x1c, 0x91, 0x8b, 0xff, 0x47, 0x80, 0xb9, 0x50, 0x9b, 0xf8, 0x9c, 0x58, 0x87, - 0xac, 0x6f, 0x0e, 0xec, 0x92, 0x70, 0xc1, 0xe0, 0xc9, 0x20, 0x33, 0x92, 0xe1, 0xb2, 0xfb, 0x58, - 0x9a, 0x53, 0xde, 0x17, 0x9b, 0xb8, 0xa0, 0xd8, 0x4b, 0xbe, 0x9c, 0xb5, 0xc0, 0x03, 0xbc, 0x49, - 0x92, 0x1c, 0x6b, 0x92, 0x88, 0xbf, 0x2d, 0x40, 0x91, 0x3e, 0xe0, 0x09, 0xc6, 0x5a, 0x2c, 0xd6, - 0xc9, 0x0d, 0x4a, 0x4e, 0x8c, 0x7f, 0x6e, 0x24, 0xf4, 0x1d, 0x8c, 0x64, 0xf8, 0x3b, 0x18, 0xe2, - 0x0f, 0x04, 0xc8, 0x7b, 0x35, 0x64, 0x5f, 0x89, 0x1b, 0x91, 0x9c, 0xf2, 0xcd, 0xbe, 0x84, 0xe6, - 0x66, 0xe1, 0x18, 0xeb, 0xc3, 0x75, 0xc1, 0x2c, 0x1c, 0xec, 0x0b, 0x5e, 0x7f, 0xdf, 0x1d, 0x39, - 0xa4, 0x8a, 0x15, 0x3f, 0xdb, 0xc2, 0x1b, 0x1c, 0xa1, 0x91, 0xe8, 0x07, 0x36, 0xcd, 0xf6, 0x21, - 0x4b, 0x7c, 0x32, 0x96, 0xd9, 0x42, 0x3c, 0x1c, 0x0a, 0xf8, 0x6e, 0x9c, 0xd6, 0x6c, 0xd0, 0x4f, - 0x6f, 0xb2, 0xdf, 0xb6, 0xf8, 0x24, 0xa0, 0x40, 0xda, 0xf9, 0x44, 0x4b, 0x63, 0x99, 0x52, 0x57, - 0x4b, 0x6c, 0xac, 0xfc, 0x49, 0xb0, 0x27, 0xaa, 0x87, 0x04, 0x85, 0x3d, 0x82, 0xe4, 0xa1, 0xd2, - 0x1e, 0x15, 0x06, 0x16, 0xea, 0x39, 0x89, 0x94, 0x46, 0x4f, 0x42, 0x49, 0x2a, 0x12, 0xc3, 0x11, - 0xc3, 0xa0, 0x4a, 0x43, 0xc9, 0x2c, 0x3e, 0x0c, 0x8f, 0xf5, 0x91, 0x8f, 0x0f, 0x0e, 0xfa, 0x8f, - 0x53, 0x3f, 0xfc, 0xc1, 0xa2, 0x20, 0x7e, 0x02, 0x48, 0xc2, 0x36, 0x76, 0x5e, 0xf4, 0x4c, 0xcb, - 0x4f, 0xf8, 0x71, 0xbb, 0xef, 0xc3, 0x20, 0x93, 0xe5, 0xec, 0x59, 0x54, 0x4a, 0x9f, 0x4b, 0x30, - 0x17, 0xe2, 0x66, 0xc6, 0x42, 0xfc, 0x10, 0xae, 0x3e, 0x35, 0x6d, 0x5b, 0xef, 0x12, 0xe8, 0x49, - 0x67, 0x25, 0x59, 0x1a, 0x3c, 0xf3, 0x98, 0xee, 0xd2, 0x4d, 0x08, 0x83, 0x99, 0x91, 0x8c, 0xe4, - 0x5d, 0x8b, 0x7f, 0x28, 0xc0, 0x95, 0x41, 0x4e, 0xa6, 0xe5, 0xa8, 0x13, 0x7f, 0xd3, 0xaa, 0xe9, - 0xe7, 0x77, 0x3b, 0x7f, 0xb4, 0xba, 0xc5, 0x89, 0x8b, 0xc9, 0x9f, 0x29, 0x77, 0x14, 0x6a, 0x3e, - 0xf8, 0xe9, 0xe3, 0x3c, 0x27, 0x6f, 0x32, 0xaa, 0x6f, 0x49, 0x52, 0xe3, 0x59, 0x92, 0x26, 0x14, - 0xd6, 0x4d, 0xdd, 0x20, 0x9e, 0xac, 0xdb, 0xde, 0x55, 0xc8, 0xef, 0xea, 0x86, 0x62, 0x9d, 0xc8, - 0xee, 0xa1, 0x6e, 0x36, 0x50, 0x16, 0xa2, 0x2a, 0xcb, 0x4a, 0x48, 0x39, 0xc6, 0xc1, 0x2f, 0xc5, - 0x1f, 0x09, 0x50, 0xf4, 0xc5, 0x72, 0x8b, 0xfc, 0x2e, 0x80, 0xda, 0xee, 0xd9, 0x0e, 0xb6, 0xdc, - 0x5e, 0x9a, 0x61, 0xd1, 0xdb, 0x15, 0x46, 0xad, 0xaf, 0x49, 0x19, 0x5e, 0xa0, 0xae, 0xa1, 0x1b, - 0xe1, 0xe4, 0x08, 0x93, 0x65, 0x38, 0x1b, 0x48, 0x89, 0x40, 0xba, 0xdd, 0x76, 0x4c, 0xcb, 0xc3, - 0x4c, 0xbc, 0xdb, 0xdd, 0x54, 0xea, 0xf4, 0x4c, 0x2f, 0x29, 0xb7, 0x0a, 0x79, 0xb2, 0xdc, 0x1f, - 0x62, 0xaf, 0x49, 0xa9, 0xf3, 0x9b, 0xc4, 0x38, 0xdc, 0x26, 0xfd, 0x1e, 0x71, 0x52, 0x59, 0x6f, - 0x78, 0x3d, 0x3c, 0xc2, 0xa2, 0xfd, 0x4c, 0x30, 0x25, 0xe1, 0x78, 0x49, 0x1b, 0xa9, 0x27, 0xf2, - 0x2d, 0x48, 0xbb, 0x9f, 0x2e, 0xe6, 0x13, 0xe4, 0xea, 0x32, 0xfb, 0xb6, 0xf1, 0xb2, 0xfb, 0x6d, - 0xe3, 0xe5, 0x35, 0x5e, 0x80, 0x99, 0xf1, 0xef, 0xff, 0xf7, 0x45, 0x41, 0xf2, 0x98, 0xee, 0x37, - 0xc8, 0x08, 0x1f, 0x58, 0x85, 0x51, 0x1e, 0x20, 0xf0, 0x0d, 0x19, 0xfe, 0x35, 0xdb, 0xd5, 0x35, - 0x79, 0x67, 0xab, 0xf2, 0x7c, 0x73, 0xb3, 0xde, 0x6c, 0x56, 0xd7, 0x8a, 0x02, 0x2a, 0xc2, 0x4c, - 0xe8, 0x0b, 0x34, 0x09, 0xf6, 0x7d, 0xdb, 0xfb, 0x3f, 0x03, 0xe0, 0x7f, 0xcc, 0x8a, 0xc8, 0xda, - 0xa8, 0x7e, 0x26, 0xbf, 0x5c, 0x7d, 0xb6, 0x53, 0x6d, 0x14, 0x27, 0x10, 0x82, 0x7c, 0x79, 0xb5, - 0x59, 0xa9, 0xc9, 0x52, 0xb5, 0xb1, 0xfd, 0x7c, 0xab, 0x51, 0x75, 0xbf, 0x8b, 0x7b, 0x7f, 0x0d, - 0x66, 0x82, 0xe9, 0x6e, 0xd0, 0x1c, 0x14, 0x2a, 0xb5, 0x6a, 0x65, 0x43, 0x7e, 0x59, 0x5f, 0x95, - 0x5f, 0xec, 0x54, 0x77, 0xaa, 0xc5, 0x09, 0x5a, 0x35, 0x4a, 0x7c, 0xb2, 0xf3, 0xec, 0x59, 0x51, - 0x40, 0x05, 0xc8, 0xb2, 0x6b, 0xfa, 0xb5, 0x9a, 0x62, 0xe2, 0xfe, 0x26, 0x64, 0x03, 0x69, 0x6d, - 0xc9, 0xe3, 0xb6, 0x77, 0x1a, 0x35, 0xb9, 0x59, 0xdf, 0xac, 0x36, 0x9a, 0xab, 0x9b, 0xdb, 0x4c, - 0x06, 0xa5, 0xad, 0x96, 0x9f, 0x4b, 0xcd, 0xa2, 0xe0, 0x5d, 0x37, 0x9f, 0xef, 0x54, 0x6a, 0x6e, - 0x33, 0xc4, 0x54, 0x3a, 0x59, 0x4c, 0xde, 0xff, 0x1b, 0x02, 0x5c, 0x19, 0x92, 0xfa, 0x05, 0x65, - 0x61, 0x7a, 0xc7, 0xa0, 0x39, 0x3e, 0x8b, 0x13, 0x28, 0x17, 0xc8, 0xfe, 0x52, 0x14, 0x50, 0x9a, - 0xe5, 0xdf, 0x28, 0x26, 0xd0, 0x14, 0x24, 0x1a, 0x8f, 0x8a, 0x49, 0x52, 0xd3, 0x40, 0xf2, 0x94, - 0x62, 0x0a, 0x65, 0x78, 0xda, 0x86, 0xe2, 0x24, 0x9a, 0xf1, 0xb3, 0x27, 0x14, 0xa7, 0x88, 0x28, - 0x2f, 0x0b, 0x41, 0x71, 0xfa, 0xfe, 0x75, 0x08, 0x9c, 0xf4, 0x46, 0x00, 0x53, 0xcf, 0x14, 0x07, - 0xdb, 0x4e, 0x71, 0x02, 0x4d, 0x43, 0x72, 0xb5, 0xdd, 0x2e, 0x0a, 0x0f, 0xff, 0x6d, 0x0a, 0xd2, - 0xee, 0x57, 0x59, 0xd0, 0x33, 0x98, 0x64, 0x5b, 0xcf, 0x8b, 0xc3, 0x3d, 0x7b, 0x3a, 0x79, 0x17, - 0x96, 0xce, 0x73, 0xfd, 0xc5, 0x09, 0xf4, 0xd7, 0x20, 0x1b, 0xf0, 0x98, 0xd0, 0xd0, 0xed, 0xb3, - 0x90, 0x97, 0xb8, 0x70, 0xfb, 0xbc, 0x62, 0x9e, 0xfc, 0x57, 0x90, 0xf1, 0x2c, 0x38, 0xba, 0x31, - 0xca, 0xbe, 0xbb, 0xb2, 0x47, 0x2f, 0x02, 0x64, 0xae, 0x89, 0x13, 0xef, 0x0b, 0xc8, 0x02, 0x34, - 0x68, 0x6c, 0x51, 0x54, 0x44, 0xc2, 0x50, 0x6b, 0xbe, 0x70, 0x7f, 0xac, 0xd2, 0xfe, 0x33, 0x89, - 0xb2, 0xfc, 0x15, 0x23, 0x5a, 0x59, 0x03, 0xeb, 0x51, 0xb4, 0xb2, 0x22, 0x16, 0x9e, 0x09, 0xf4, - 0x02, 0x52, 0xc4, 0x52, 0xa2, 0x28, 0x1f, 0xb2, 0xcf, 0x32, 0x2f, 0xdc, 0x18, 0x59, 0xc6, 0x15, - 0x59, 0xbe, 0xf7, 0xc3, 0xbf, 0xb8, 0x36, 0xf1, 0xc3, 0xb3, 0x6b, 0xc2, 0x8f, 0xce, 0xae, 0x09, - 0x7f, 0x76, 0x76, 0x4d, 0xf8, 0xf3, 0xb3, 0x6b, 0xc2, 0xaf, 0xff, 0xf8, 0xda, 0xc4, 0x8f, 0x7e, - 0x7c, 0x6d, 0xe2, 0xcf, 0x7e, 0x7c, 0x6d, 0xe2, 0xf3, 0x69, 0xce, 0xbd, 0x3b, 0x45, 0x8d, 0xca, - 0xa3, 0xff, 0x1f, 0x00, 0x00, 0xff, 0xff, 0x9c, 0xe1, 0xc2, 0x1d, 0x6b, 0x7e, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_4d4da7f10ee893f6) } + +var fileDescriptor_api_4d4da7f10ee893f6 = []byte{ + // 8188 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x6b, 0x6c, 0x23, 0x59, + 0x76, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0x50, 0x7c, 0xe8, 0xaa, 0x1f, 0x6c, 0xce, 0x4c, 0xab, 0xbb, + 0xfa, 0xdd, 0x3b, 0x23, 0x4d, 0x77, 0xef, 0x78, 0xc6, 0xd3, 0xe3, 0x59, 0x4b, 0x14, 0xbb, 0x49, + 0xa9, 0xa5, 0x56, 0x17, 0xa9, 0x6e, 0xcf, 0x78, 0x9d, 0xda, 0x52, 0xd5, 0x15, 0x55, 0x2b, 0xb2, + 0x8a, 0x5d, 0x55, 0xd4, 0x63, 0x80, 0x00, 0x71, 0x6c, 0x24, 0x06, 0x02, 0x2c, 0xf6, 0x87, 0x83, + 0xac, 0xe1, 0x24, 0x5e, 0xc7, 0x71, 0x1c, 0x20, 0xc8, 0x03, 0x48, 0x10, 0x07, 0x41, 0x62, 0xff, + 0x31, 0x90, 0x45, 0xe0, 0x20, 0xeb, 0x5f, 0x31, 0x02, 0x44, 0xb1, 0xb5, 0xf9, 0x97, 0x20, 0x08, + 0xf2, 0x27, 0xc0, 0xfc, 0x08, 0x82, 0xfb, 0xa8, 0x17, 0x59, 0x7c, 0xa8, 0xb7, 0x06, 0x19, 0xc0, + 0x7f, 0x24, 0xd6, 0xa9, 0x7b, 0x4e, 0xdd, 0x7b, 0xee, 0xbd, 0xe7, 0x9e, 0xef, 0xd6, 0xb9, 0xa7, + 0x60, 0xde, 0x32, 0x15, 0x75, 0xbf, 0xbb, 0xbb, 0xac, 0x74, 0xf5, 0xa5, 0xae, 0x65, 0x3a, 0x26, + 0x9a, 0x57, 0x4d, 0xf5, 0x80, 0x92, 0x97, 0xf8, 0xcd, 0xf2, 0xfd, 0x83, 0xc3, 0xe5, 0x83, 0x43, + 0x1b, 0x5b, 0x87, 0xd8, 0x5a, 0x56, 0x4d, 0x43, 0xed, 0x59, 0x16, 0x36, 0xd4, 0x93, 0xe5, 0xb6, + 0xa9, 0x1e, 0xd0, 0x3f, 0xba, 0xd1, 0x62, 0xec, 0x65, 0xe4, 0x4a, 0xd4, 0x14, 0x47, 0xe1, 0xb4, + 0x0b, 0x2e, 0x0d, 0x5b, 0x96, 0x69, 0xd9, 0x9c, 0x7a, 0xc9, 0xa5, 0x76, 0xb0, 0xa3, 0x04, 0x4a, + 0xbf, 0x65, 0x3b, 0xa6, 0xa5, 0xb4, 0xf0, 0x32, 0x36, 0x5a, 0xba, 0x81, 0x49, 0x81, 0x43, 0x55, + 0xe5, 0x37, 0xdf, 0x8e, 0xbc, 0xf9, 0x88, 0xdf, 0x2d, 0xf5, 0x1c, 0xbd, 0xbd, 0xbc, 0xdf, 0x56, + 0x97, 0x1d, 0xbd, 0x83, 0x6d, 0x47, 0xe9, 0x74, 0xf9, 0x9d, 0xfb, 0xf4, 0x8e, 0x63, 0x29, 0xaa, + 0x6e, 0xb4, 0xdc, 0xff, 0xdd, 0xdd, 0x65, 0x0b, 0xab, 0xa6, 0xa5, 0x61, 0x4d, 0xb6, 0xbb, 0x8a, + 0xe1, 0x56, 0xb7, 0x65, 0xb6, 0x4c, 0xfa, 0x73, 0x99, 0xfc, 0xe2, 0xd4, 0xab, 0x2d, 0xd3, 0x6c, + 0xb5, 0xf1, 0x32, 0xbd, 0xda, 0xed, 0xed, 0x2d, 0x6b, 0x3d, 0x4b, 0x71, 0x74, 0x93, 0x73, 0x89, + 0xff, 0x52, 0x80, 0x9c, 0x84, 0x5f, 0xf7, 0xb0, 0xed, 0xd4, 0xb0, 0xa2, 0x61, 0x0b, 0x5d, 0x81, + 0xe4, 0x01, 0x3e, 0x29, 0x25, 0xaf, 0x09, 0x77, 0xe7, 0x56, 0x67, 0xbf, 0x3c, 0x5d, 0x4c, 0x6e, + 0xe0, 0x13, 0x89, 0xd0, 0xd0, 0x35, 0x98, 0xc5, 0x86, 0x26, 0x93, 0xdb, 0xa9, 0xf0, 0xed, 0x19, + 0x6c, 0x68, 0x1b, 0xf8, 0x04, 0x7d, 0x1b, 0xd2, 0x36, 0x91, 0x66, 0xa8, 0xb8, 0x34, 0x7d, 0x4d, + 0xb8, 0x3b, 0xbd, 0xfa, 0xf3, 0x5f, 0x9e, 0x2e, 0x7e, 0xd2, 0xd2, 0x9d, 0xfd, 0xde, 0xee, 0x92, + 0x6a, 0x76, 0x96, 0xbd, 0x7e, 0xd2, 0x76, 0xfd, 0xdf, 0xcb, 0xdd, 0x83, 0xd6, 0x72, 0xbf, 0x8e, + 0x96, 0x9a, 0xc7, 0x46, 0x03, 0xbf, 0x96, 0x3c, 0x89, 0xeb, 0xa9, 0xb4, 0x50, 0x4c, 0xac, 0xa7, + 0xd2, 0x89, 0x62, 0x52, 0xfc, 0x9d, 0x24, 0xe4, 0x25, 0x6c, 0x77, 0x4d, 0xc3, 0xc6, 0xbc, 0xe6, + 0xef, 0x43, 0xd2, 0x39, 0x36, 0x68, 0xcd, 0xb3, 0x0f, 0xaf, 0x2e, 0x0d, 0x8c, 0x88, 0xa5, 0xa6, + 0xa5, 0x18, 0xb6, 0xa2, 0x92, 0xe6, 0x4b, 0xa4, 0x28, 0xfa, 0x08, 0xb2, 0x16, 0xb6, 0x7b, 0x1d, + 0x4c, 0x15, 0x49, 0x1b, 0x95, 0x7d, 0x78, 0x39, 0x82, 0xb3, 0xd1, 0x55, 0x0c, 0x09, 0x58, 0x59, + 0xf2, 0x1b, 0x5d, 0x81, 0xb4, 0xd1, 0xeb, 0x10, 0x55, 0xd8, 0xb4, 0xa1, 0x49, 0x69, 0xd6, 0xe8, + 0x75, 0x36, 0xf0, 0x89, 0x8d, 0x7e, 0x01, 0x2e, 0x69, 0xb8, 0x6b, 0x61, 0x55, 0x71, 0xb0, 0x26, + 0x5b, 0x8a, 0xd1, 0xc2, 0xb2, 0x6e, 0xec, 0x99, 0x76, 0x69, 0xe6, 0x5a, 0xf2, 0x6e, 0xf6, 0xe1, + 0xdb, 0x11, 0xf2, 0x25, 0x52, 0xaa, 0x6e, 0xec, 0x99, 0xab, 0xa9, 0x1f, 0x9d, 0x2e, 0x4e, 0x49, + 0x17, 0x7c, 0x09, 0xde, 0x2d, 0x1b, 0x35, 0x20, 0xc7, 0xab, 0x6b, 0x61, 0xc5, 0x36, 0x8d, 0xd2, + 0xec, 0x35, 0xe1, 0x6e, 0xfe, 0xe1, 0x52, 0x94, 0xc0, 0x90, 0x6a, 0xc8, 0x65, 0xaf, 0x83, 0x25, + 0xca, 0x25, 0xcd, 0x59, 0x81, 0x2b, 0xf4, 0x16, 0x64, 0x48, 0x4b, 0x76, 0x4f, 0x1c, 0x6c, 0x97, + 0xd2, 0xb4, 0x29, 0xa4, 0x69, 0xab, 0xe4, 0x5a, 0xfc, 0x14, 0xe6, 0x82, 0xac, 0x08, 0x41, 0x5e, + 0xaa, 0x36, 0x76, 0x36, 0xab, 0xf2, 0xce, 0xd6, 0xc6, 0xd6, 0xf3, 0x57, 0x5b, 0xc5, 0x29, 0x74, + 0x01, 0x8a, 0x9c, 0xb6, 0x51, 0xfd, 0x4c, 0x7e, 0x56, 0xdf, 0xac, 0x37, 0x8b, 0x42, 0x39, 0xf5, + 0x6b, 0xbf, 0x73, 0x75, 0x4a, 0xdc, 0x06, 0x78, 0x8a, 0x1d, 0x3e, 0xc0, 0xd0, 0x2a, 0xcc, 0xec, + 0xd3, 0xfa, 0x94, 0x04, 0xaa, 0xe9, 0x6b, 0x91, 0x15, 0x0f, 0x0c, 0xc6, 0xd5, 0x34, 0xd1, 0xc6, + 0x8f, 0x4f, 0x17, 0x05, 0x89, 0x73, 0x8a, 0x7f, 0x28, 0x40, 0x96, 0x8a, 0x64, 0xed, 0x43, 0x95, + 0x3e, 0x99, 0xd7, 0xc7, 0x2a, 0x63, 0x50, 0x28, 0x5a, 0x82, 0xe9, 0x43, 0xa5, 0xdd, 0xc3, 0xa5, + 0x04, 0x95, 0x51, 0x8a, 0x90, 0xf1, 0x92, 0xdc, 0x97, 0x58, 0x31, 0xf4, 0x18, 0xe6, 0x74, 0xc3, + 0xc1, 0x86, 0x23, 0x33, 0xb6, 0xe4, 0x18, 0xb6, 0x2c, 0x2b, 0x4d, 0x2f, 0xc4, 0x7f, 0x21, 0x00, + 0x6c, 0xf7, 0xe2, 0x54, 0x0a, 0xfa, 0xe6, 0x84, 0xf5, 0xe7, 0xa3, 0x8b, 0xb7, 0xe2, 0x12, 0xcc, + 0xe8, 0x46, 0x5b, 0x37, 0x58, 0xfd, 0xd3, 0x12, 0xbf, 0x42, 0x17, 0x60, 0x7a, 0xb7, 0xad, 0x1b, + 0x1a, 0x9d, 0x0f, 0x69, 0x89, 0x5d, 0x88, 0x12, 0x64, 0x69, 0xad, 0x63, 0xd4, 0xbb, 0x78, 0x9a, + 0x80, 0x8b, 0x15, 0xd3, 0xd0, 0x74, 0x32, 0x25, 0x95, 0xf6, 0xd7, 0x42, 0x2b, 0xeb, 0x10, 0x98, + 0x7c, 0x32, 0x3e, 0xee, 0x4e, 0xd8, 0xc7, 0xc8, 0xe7, 0xaa, 0x1e, 0x77, 0x29, 0x2d, 0x5a, 0x93, + 0xe8, 0x9b, 0x70, 0x59, 0x69, 0xb7, 0xcd, 0x23, 0x59, 0xdf, 0x93, 0x35, 0x13, 0xdb, 0xb2, 0x61, + 0x3a, 0x32, 0x3e, 0xd6, 0x6d, 0x87, 0x9a, 0x92, 0xb4, 0xb4, 0x40, 0x6f, 0xd7, 0xf7, 0xd6, 0x4c, + 0x6c, 0x6f, 0x99, 0x4e, 0x95, 0xdc, 0x22, 0xf3, 0x94, 0x54, 0x86, 0xcd, 0xd3, 0x19, 0x62, 0x7e, + 0xa5, 0x34, 0x3e, 0xee, 0xd2, 0x79, 0x1a, 0xe8, 0xca, 0xd9, 0x60, 0x57, 0x8a, 0xbf, 0x04, 0x97, + 0xfa, 0xf5, 0x1b, 0x67, 0xff, 0xfd, 0xb1, 0x00, 0xf9, 0xba, 0xa1, 0x3b, 0x5f, 0x8b, 0x8e, 0xf3, + 0x94, 0x9d, 0x0c, 0x2a, 0xfb, 0x3e, 0x14, 0xf7, 0x14, 0xbd, 0xfd, 0xdc, 0x68, 0x9a, 0x9d, 0x5d, + 0xdb, 0x31, 0x0d, 0x6c, 0xf3, 0xde, 0x18, 0xa0, 0x8b, 0x2f, 0xa1, 0xe0, 0xb5, 0x26, 0x4e, 0x35, + 0x39, 0x50, 0xac, 0x1b, 0xaa, 0x85, 0x3b, 0xd8, 0x88, 0x55, 0x4f, 0x6f, 0x43, 0x46, 0x77, 0xe5, + 0x52, 0x5d, 0x25, 0x25, 0x9f, 0x20, 0xf6, 0x60, 0x3e, 0xf0, 0xd4, 0x38, 0xcd, 0x25, 0x59, 0x32, + 0xf0, 0x91, 0xec, 0xf7, 0x11, 0x59, 0x32, 0xf0, 0x11, 0x33, 0x6f, 0x0d, 0xc8, 0xad, 0xe1, 0x36, + 0x76, 0x70, 0x9c, 0x56, 0x7f, 0x07, 0xf2, 0xae, 0xd0, 0x38, 0x3b, 0xe6, 0x6f, 0x09, 0x80, 0xb8, + 0x5c, 0xb2, 0xca, 0xc6, 0xd9, 0x37, 0x8b, 0xc4, 0xb5, 0x70, 0x7a, 0x96, 0xc1, 0x7c, 0x04, 0x36, + 0x26, 0x81, 0x91, 0xa8, 0x9b, 0xe0, 0x4f, 0xd9, 0x54, 0x70, 0xca, 0x72, 0xf7, 0xe6, 0x08, 0x16, + 0x42, 0x15, 0x8b, 0xb7, 0xfb, 0x52, 0xb4, 0x4e, 0x89, 0x6b, 0xc9, 0xa0, 0x0f, 0x47, 0x89, 0xe2, + 0x0f, 0x04, 0x98, 0xaf, 0xb4, 0xb1, 0x62, 0xc5, 0xae, 0x91, 0x6f, 0x41, 0x5a, 0xc3, 0x8a, 0x46, + 0x9b, 0xcc, 0x26, 0xf6, 0x3b, 0x01, 0x29, 0xc4, 0xd3, 0x5d, 0xda, 0x6f, 0xab, 0x4b, 0x4d, 0xd7, + 0x07, 0xe6, 0xb3, 0xdb, 0x63, 0x12, 0x3f, 0x03, 0x14, 0xac, 0x59, 0x9c, 0x03, 0xe1, 0xef, 0x0a, + 0x80, 0x24, 0x7c, 0x88, 0x2d, 0x27, 0xf6, 0x66, 0xaf, 0x41, 0xd6, 0x51, 0xac, 0x16, 0x76, 0x64, + 0xe2, 0xdd, 0x9f, 0xa7, 0xe5, 0xc0, 0xf8, 0x08, 0x59, 0xfc, 0x1c, 0x16, 0x42, 0xf5, 0x8b, 0xb3, + 0xf1, 0xff, 0x4b, 0x80, 0x6c, 0x43, 0x55, 0x8c, 0x38, 0x5b, 0xfd, 0x29, 0x64, 0x6d, 0x55, 0x31, + 0xe4, 0x3d, 0xd3, 0xea, 0x28, 0x0e, 0x1d, 0xe2, 0xf9, 0x50, 0xab, 0x3d, 0xcf, 0x5a, 0x55, 0x8c, + 0x27, 0xb4, 0x90, 0x04, 0xb6, 0xf7, 0x1b, 0xbd, 0x80, 0xec, 0x01, 0x3e, 0x91, 0x39, 0x4a, 0xa3, + 0xeb, 0x62, 0xfe, 0xe1, 0xfb, 0x01, 0xfe, 0x83, 0xc3, 0x25, 0x17, 0xdc, 0x2d, 0x05, 0xc0, 0xdd, + 0x12, 0xe1, 0x58, 0x6a, 0x38, 0x16, 0x36, 0x5a, 0xce, 0xbe, 0x04, 0x07, 0xf8, 0xe4, 0x19, 0x93, + 0xc1, 0x26, 0xd6, 0x7a, 0x2a, 0x9d, 0x2c, 0xa6, 0xc4, 0xff, 0x23, 0xc0, 0x1c, 0x6b, 0x72, 0x9c, + 0x13, 0xeb, 0x03, 0x48, 0x59, 0xe6, 0x11, 0x9b, 0x58, 0xd9, 0x87, 0x6f, 0x45, 0x88, 0xd8, 0xc0, + 0x27, 0xc1, 0x95, 0x8b, 0x16, 0x47, 0xab, 0xc0, 0xfd, 0x43, 0x99, 0x72, 0x27, 0x27, 0xe5, 0x06, + 0xc6, 0x25, 0x11, 0x19, 0x77, 0xa0, 0xb0, 0xab, 0x38, 0xea, 0xbe, 0x6c, 0xf1, 0x4a, 0x92, 0x55, + 0x2e, 0x79, 0x77, 0x4e, 0xca, 0x53, 0xb2, 0x5b, 0x75, 0x9b, 0xb4, 0x9c, 0x8d, 0x74, 0x1b, 0xff, + 0x05, 0xeb, 0xf3, 0xff, 0x2b, 0xf0, 0x39, 0xe4, 0xb6, 0xfc, 0x2f, 0x5a, 0xd7, 0xff, 0x46, 0x02, + 0x2e, 0x57, 0xf6, 0xb1, 0x7a, 0x50, 0x31, 0x0d, 0x5b, 0xb7, 0x1d, 0xa2, 0xbb, 0x38, 0xfb, 0xff, + 0x2d, 0xc8, 0x1c, 0xe9, 0xce, 0xbe, 0xac, 0xe9, 0x7b, 0x7b, 0xd4, 0xce, 0xa5, 0xa5, 0x34, 0x21, + 0xac, 0xe9, 0x7b, 0x7b, 0xe8, 0x11, 0xa4, 0x3a, 0xa6, 0xc6, 0xdc, 0xe8, 0xfc, 0xc3, 0xc5, 0x08, + 0xf1, 0xb4, 0x6a, 0x76, 0xaf, 0xb3, 0x69, 0x6a, 0x58, 0xa2, 0x85, 0xd1, 0x55, 0x00, 0x95, 0x50, + 0xbb, 0xa6, 0x6e, 0x38, 0x7c, 0x9d, 0x0c, 0x50, 0x50, 0x0d, 0x32, 0x0e, 0xb6, 0x3a, 0xba, 0xa1, + 0x38, 0xb8, 0x34, 0x4d, 0x95, 0x77, 0x33, 0xb2, 0xe2, 0xdd, 0xb6, 0xae, 0x2a, 0x6b, 0xd8, 0x56, + 0x2d, 0xbd, 0xeb, 0x98, 0x16, 0xd7, 0xa2, 0xcf, 0x2c, 0x7e, 0x2f, 0x05, 0xa5, 0x41, 0xdd, 0xc4, + 0x39, 0x42, 0xb6, 0x61, 0x86, 0xe0, 0xee, 0xb6, 0xc3, 0xc7, 0xc8, 0xc3, 0x61, 0x2a, 0x88, 0xa8, + 0x01, 0xc5, 0xef, 0x6d, 0x87, 0x57, 0x9b, 0xcb, 0x29, 0xff, 0x1b, 0x01, 0x66, 0xd8, 0x0d, 0xf4, + 0x00, 0xd2, 0x7c, 0xa3, 0x41, 0xa3, 0x75, 0x4c, 0xae, 0x5e, 0x3a, 0x3b, 0x5d, 0x9c, 0x65, 0x7b, + 0x07, 0x6b, 0x5f, 0xfa, 0x3f, 0xa5, 0x59, 0x5a, 0xae, 0xae, 0x91, 0xde, 0xb2, 0x1d, 0xc5, 0x72, + 0xe8, 0x76, 0x4e, 0x82, 0xe1, 0x09, 0x4a, 0xd8, 0xc0, 0x27, 0x68, 0x1d, 0x66, 0x6c, 0x47, 0x71, + 0x7a, 0x36, 0xef, 0xaf, 0x73, 0x55, 0xb6, 0x41, 0x39, 0x25, 0x2e, 0x81, 0x38, 0x3a, 0x1a, 0x76, + 0x14, 0xbd, 0x4d, 0x3b, 0x30, 0x23, 0xf1, 0x2b, 0xf1, 0x37, 0x05, 0x98, 0x61, 0x45, 0xd1, 0x65, + 0x58, 0x90, 0x56, 0xb6, 0x9e, 0x56, 0xe5, 0xfa, 0xd6, 0x5a, 0xb5, 0x59, 0x95, 0x36, 0xeb, 0x5b, + 0x2b, 0xcd, 0x6a, 0x71, 0x0a, 0x5d, 0x02, 0xe4, 0xde, 0xa8, 0x3c, 0xdf, 0x6a, 0xd4, 0x1b, 0xcd, + 0xea, 0x56, 0xb3, 0x28, 0xd0, 0x3d, 0x07, 0x4a, 0x0f, 0x50, 0x13, 0xe8, 0x26, 0x5c, 0xeb, 0xa7, + 0xca, 0x8d, 0xe6, 0x4a, 0xb3, 0x21, 0x57, 0x1b, 0xcd, 0xfa, 0xe6, 0x4a, 0xb3, 0xba, 0x56, 0x4c, + 0x8e, 0x28, 0x45, 0x1e, 0x22, 0x49, 0xd5, 0x4a, 0xb3, 0x98, 0x12, 0x1d, 0xb8, 0x28, 0x61, 0xd5, + 0xec, 0x74, 0x7b, 0x0e, 0x26, 0xb5, 0xb4, 0xe3, 0x9c, 0x29, 0x97, 0x61, 0x56, 0xb3, 0x4e, 0x64, + 0xab, 0x67, 0xf0, 0x79, 0x32, 0xa3, 0x59, 0x27, 0x52, 0xcf, 0x10, 0xff, 0x99, 0x00, 0x97, 0xfa, + 0x1f, 0x1b, 0xe7, 0x20, 0x7c, 0x01, 0x59, 0x45, 0xd3, 0xb0, 0x26, 0x6b, 0xb8, 0xed, 0x28, 0xdc, + 0x19, 0xb9, 0x1f, 0x90, 0xc4, 0x37, 0xe1, 0x96, 0xbc, 0x4d, 0xb8, 0xcd, 0x97, 0x95, 0x0a, 0xad, + 0xc8, 0x1a, 0xe1, 0x70, 0xcd, 0x0f, 0x15, 0x42, 0x29, 0xe2, 0xff, 0x48, 0x41, 0xae, 0x6a, 0x68, + 0xcd, 0xe3, 0x58, 0xd7, 0x92, 0x4b, 0x30, 0xa3, 0x9a, 0x9d, 0x8e, 0xee, 0xb8, 0x0a, 0x62, 0x57, + 0xe8, 0x67, 0x03, 0x4e, 0x64, 0x72, 0x02, 0x57, 0xca, 0x77, 0x1f, 0xd1, 0x77, 0xe0, 0x32, 0xb1, + 0x9a, 0x96, 0xa1, 0xb4, 0x65, 0x26, 0x4d, 0x76, 0x2c, 0xbd, 0xd5, 0xc2, 0x16, 0xdf, 0xf8, 0xbb, + 0x1b, 0x51, 0xcf, 0x3a, 0xe7, 0xa8, 0x50, 0x86, 0x26, 0x2b, 0x2f, 0x5d, 0xd4, 0xa3, 0xc8, 0xe8, + 0x13, 0x00, 0xb2, 0x14, 0xd1, 0xcd, 0x44, 0x9b, 0xdb, 0xa3, 0x61, 0xbb, 0x89, 0xae, 0x09, 0x22, + 0x0c, 0xe4, 0xda, 0x46, 0xcb, 0x04, 0x31, 0xbc, 0xee, 0xe9, 0x16, 0x96, 0x1f, 0x74, 0x55, 0x0a, + 0xf1, 0xd3, 0xab, 0xf9, 0xb3, 0xd3, 0x45, 0x90, 0x18, 0xf9, 0xc1, 0x76, 0x85, 0x20, 0x08, 0xf6, + 0xbb, 0xab, 0xa2, 0x57, 0x70, 0x2f, 0xb0, 0x53, 0x41, 0x56, 0x5e, 0xde, 0x2c, 0xc5, 0x91, 0xf7, + 0xf5, 0xd6, 0x3e, 0xb6, 0x64, 0x6f, 0x43, 0x99, 0xee, 0xec, 0xa5, 0xa5, 0x9b, 0x3e, 0x43, 0x45, + 0x31, 0x58, 0xed, 0x57, 0x9c, 0x1a, 0x2d, 0xec, 0xe9, 0x8c, 0x28, 0xbf, 0x6b, 0xea, 0xb6, 0x69, + 0x94, 0x32, 0x4c, 0xf9, 0xec, 0x0a, 0xdd, 0x83, 0xa2, 0x73, 0x6c, 0xc8, 0xfb, 0x58, 0xb1, 0x9c, + 0x5d, 0xac, 0x38, 0x64, 0x95, 0x06, 0x5a, 0xa2, 0xe0, 0x1c, 0x1b, 0xb5, 0x00, 0x19, 0xbd, 0x80, + 0xa2, 0x6e, 0xc8, 0x7b, 0x6d, 0xbd, 0xb5, 0xef, 0xc8, 0x47, 0x96, 0xee, 0x60, 0xbb, 0x34, 0x4f, + 0x15, 0x12, 0x35, 0x6e, 0x1b, 0x7c, 0x87, 0x57, 0x7b, 0x45, 0x4a, 0x72, 0xd5, 0xe4, 0x75, 0xe3, + 0x09, 0xe5, 0xa7, 0x44, 0x7b, 0x3d, 0x95, 0x9e, 0x2d, 0xa6, 0xc5, 0xff, 0x22, 0x40, 0xde, 0x1d, + 0x6e, 0x71, 0xce, 0x8c, 0xbb, 0x50, 0x34, 0x0d, 0x2c, 0x77, 0xf7, 0x15, 0x1b, 0x73, 0x3d, 0xf2, + 0x05, 0x27, 0x6f, 0x1a, 0x78, 0x9b, 0x90, 0x99, 0xba, 0xd0, 0x36, 0xcc, 0xdb, 0x8e, 0xd2, 0xd2, + 0x8d, 0x56, 0x40, 0xbd, 0xd3, 0x93, 0xbb, 0xf5, 0x45, 0xce, 0xed, 0xd1, 0x43, 0x5e, 0xca, 0x9f, + 0x08, 0x30, 0xbf, 0xa2, 0x75, 0x74, 0xa3, 0xd1, 0x6d, 0xeb, 0xb1, 0xee, 0x16, 0xdc, 0x84, 0x8c, + 0x4d, 0x64, 0xfa, 0x06, 0xdf, 0xc7, 0x7e, 0x69, 0x7a, 0x87, 0x58, 0xfe, 0x67, 0x50, 0xc0, 0xc7, + 0x5d, 0x9d, 0xbd, 0x24, 0x60, 0x90, 0x25, 0x35, 0x79, 0xdb, 0xf2, 0x3e, 0x2f, 0xb9, 0xc5, 0xdb, + 0xf4, 0x19, 0xa0, 0x60, 0x93, 0xe2, 0xc4, 0x2e, 0x9f, 0xc1, 0x02, 0x15, 0xbd, 0x63, 0xd8, 0x31, + 0xeb, 0x4b, 0xfc, 0x45, 0xb8, 0x10, 0x16, 0x1d, 0x67, 0xbd, 0x5f, 0xf1, 0x5e, 0xde, 0xc4, 0x56, + 0xac, 0x70, 0xd3, 0xd3, 0x35, 0x17, 0x1c, 0x67, 0x9d, 0x7f, 0x55, 0x80, 0x2b, 0x54, 0x36, 0x7d, + 0x8f, 0xb2, 0x87, 0xad, 0x67, 0x58, 0xb1, 0x63, 0xc5, 0xca, 0x37, 0x60, 0x86, 0x61, 0x5e, 0x3a, + 0x3e, 0xa7, 0x57, 0xb3, 0xc4, 0x73, 0x69, 0x38, 0xa6, 0x45, 0x3c, 0x17, 0x7e, 0x4b, 0x54, 0xa0, + 0x1c, 0x55, 0x8b, 0x98, 0xb7, 0x03, 0xe6, 0xb9, 0xd3, 0x48, 0x86, 0x72, 0x65, 0x9f, 0xf8, 0x4c, + 0xa8, 0x0a, 0x59, 0x95, 0xfe, 0x92, 0x9d, 0x93, 0x2e, 0xa6, 0xf2, 0xf3, 0xa3, 0xfc, 0x4d, 0xc6, + 0xd6, 0x3c, 0xe9, 0x62, 0xe2, 0xb4, 0xba, 0xbf, 0x89, 0xa2, 0x02, 0x8d, 0x1c, 0xe9, 0xb1, 0xd2, + 0x79, 0x44, 0xcb, 0xba, 0xae, 0x1f, 0xd7, 0xc1, 0x3f, 0x4f, 0x72, 0x25, 0xb0, 0x67, 0xf0, 0xe2, + 0xb1, 0xfa, 0x28, 0x9f, 0x87, 0x5e, 0x63, 0x05, 0x1b, 0x9e, 0x38, 0x47, 0xc3, 0x03, 0x7b, 0xe9, + 0x3e, 0x15, 0x7d, 0x06, 0x81, 0xdd, 0x72, 0x99, 0xb5, 0xc9, 0x45, 0x3f, 0xe7, 0x51, 0xc7, 0xbc, + 0x2f, 0x85, 0xd1, 0x6d, 0x54, 0x81, 0x34, 0x3e, 0xee, 0xca, 0x1a, 0xb6, 0x55, 0x6e, 0xb8, 0xc4, + 0x61, 0xef, 0xdb, 0x06, 0xf0, 0xc0, 0x2c, 0x3e, 0xee, 0x12, 0x22, 0xda, 0x21, 0xab, 0x97, 0xeb, + 0x2a, 0xd0, 0x6a, 0xdb, 0xe3, 0xe1, 0x85, 0x3f, 0x52, 0xb8, 0xb8, 0x82, 0xe7, 0x25, 0x30, 0x11, + 0xe2, 0x0f, 0x05, 0x78, 0x2b, 0xb2, 0xd7, 0xe2, 0x5c, 0xc8, 0x3e, 0x81, 0x14, 0x6d, 0x7c, 0xe2, + 0x9c, 0x8d, 0xa7, 0x5c, 0xe2, 0xef, 0xba, 0x73, 0x5c, 0xc2, 0x6d, 0x93, 0x28, 0xf6, 0x2b, 0xd8, + 0x0f, 0x9b, 0x75, 0x3b, 0x3c, 0x71, 0xee, 0x0e, 0x77, 0x59, 0x3d, 0x23, 0xd0, 0x57, 0xcd, 0x38, + 0x8d, 0xc0, 0xf7, 0x05, 0x58, 0xf0, 0x7c, 0x9a, 0x98, 0xdd, 0xdb, 0x0f, 0x20, 0x69, 0x98, 0x47, + 0xe7, 0xd9, 0x0c, 0x24, 0xe5, 0xc9, 0x92, 0x14, 0xae, 0x51, 0x9c, 0xed, 0xfd, 0xb7, 0x09, 0xc8, + 0x3c, 0xad, 0xc4, 0xd9, 0xca, 0x4f, 0xf8, 0x46, 0x33, 0x9b, 0xd8, 0x51, 0x43, 0xd1, 0x7b, 0xde, + 0xd2, 0xd3, 0xca, 0x06, 0x3e, 0x71, 0x87, 0x22, 0xe1, 0x42, 0x2b, 0x90, 0x71, 0xf6, 0x2d, 0x6c, + 0xef, 0x9b, 0x6d, 0xed, 0x3c, 0x3e, 0x88, 0xcf, 0x55, 0xc6, 0x30, 0x4d, 0xe5, 0xba, 0x41, 0x0d, + 0x42, 0x44, 0x50, 0x03, 0x79, 0x8c, 0xe7, 0xc6, 0x25, 0xce, 0xf3, 0x98, 0x80, 0xff, 0x36, 0x5d, + 0x9c, 0x11, 0x5f, 0x00, 0x90, 0xe6, 0xc4, 0xd9, 0x25, 0x7f, 0x2d, 0x09, 0xf9, 0xed, 0x9e, 0xbd, + 0x1f, 0xf3, 0xe8, 0xab, 0x00, 0x74, 0x7b, 0x36, 0xc5, 0x07, 0xc7, 0x06, 0x6f, 0xf3, 0x98, 0x78, + 0x09, 0xb7, 0xd1, 0x8c, 0xaf, 0x79, 0x6c, 0xa0, 0x1a, 0x17, 0x82, 0x65, 0x3f, 0xe8, 0xe2, 0xc6, + 0x28, 0x24, 0xd9, 0x3c, 0x36, 0x36, 0xb1, 0x07, 0x21, 0x99, 0x24, 0x4c, 0x24, 0x7d, 0x02, 0xb3, + 0xe4, 0x42, 0x76, 0xcc, 0xf3, 0x74, 0xf3, 0x0c, 0xe1, 0x69, 0x9a, 0xe8, 0x31, 0x64, 0x18, 0x37, + 0x59, 0x9a, 0x66, 0xe8, 0xd2, 0x14, 0xd5, 0x16, 0xae, 0x46, 0xba, 0x28, 0xa5, 0x29, 0x2b, 0x59, + 0x88, 0x2e, 0xc0, 0xf4, 0x9e, 0x69, 0xa9, 0xee, 0x6b, 0x53, 0x76, 0xc1, 0xfa, 0x73, 0x3d, 0x95, + 0x4e, 0x17, 0x33, 0xeb, 0xa9, 0x74, 0xa6, 0x08, 0xe2, 0x6f, 0x0a, 0x50, 0xf0, 0x3a, 0x22, 0x4e, + 0x6b, 0x5d, 0x09, 0x69, 0xf1, 0xfc, 0x5d, 0x41, 0x14, 0x28, 0xfe, 0x3b, 0xea, 0xae, 0xa8, 0xe6, + 0x21, 0xed, 0x99, 0x38, 0x47, 0xca, 0x63, 0x16, 0x52, 0x93, 0x38, 0x6f, 0xef, 0xd2, 0xe8, 0x9a, + 0x07, 0x70, 0x41, 0xef, 0x10, 0x3b, 0xae, 0x3b, 0xed, 0x13, 0x8e, 0xa9, 0x1c, 0xec, 0xbe, 0x9f, + 0x5d, 0xf0, 0xef, 0x55, 0xdc, 0x5b, 0xe2, 0xdf, 0xa7, 0xbb, 0xd3, 0x7e, 0x4b, 0xe2, 0x54, 0x75, + 0x1d, 0x72, 0x16, 0x13, 0x4d, 0x7c, 0x8e, 0x73, 0x6a, 0x7b, 0xce, 0x63, 0x25, 0x0a, 0xff, 0xed, + 0x04, 0x14, 0x5e, 0xf4, 0xb0, 0x75, 0xf2, 0x75, 0x52, 0xf7, 0x6d, 0x28, 0x1c, 0x29, 0xba, 0x23, + 0xef, 0x99, 0x96, 0xdc, 0xeb, 0x6a, 0x8a, 0xe3, 0xc6, 0x75, 0xe4, 0x08, 0xf9, 0x89, 0x69, 0xed, + 0x50, 0x22, 0xc2, 0x80, 0x0e, 0x0c, 0xf3, 0xc8, 0x90, 0x09, 0x99, 0xa2, 0xd8, 0x63, 0x83, 0x6f, + 0x19, 0xaf, 0x7e, 0xf8, 0x9f, 0x4f, 0x17, 0x1f, 0x4d, 0x14, 0xad, 0x45, 0x23, 0xd3, 0x7a, 0x3d, + 0x5d, 0x5b, 0xda, 0xd9, 0xa9, 0xaf, 0x49, 0x45, 0x2a, 0xf2, 0x15, 0x93, 0xd8, 0x3c, 0x36, 0x6c, + 0xf1, 0x1f, 0x24, 0xa0, 0xe8, 0xeb, 0x28, 0xce, 0x8e, 0xac, 0x42, 0xf6, 0x75, 0x0f, 0x5b, 0xfa, + 0x1b, 0x74, 0x23, 0x70, 0x46, 0x62, 0x76, 0x3e, 0x87, 0xb9, 0x90, 0x06, 0x92, 0x3f, 0x9d, 0x06, + 0xb2, 0x47, 0x7e, 0xe3, 0xd1, 0x7d, 0x98, 0x77, 0x8e, 0x0d, 0x99, 0xc5, 0xe9, 0xb1, 0xd8, 0x0e, + 0x37, 0xec, 0xa0, 0xe0, 0x10, 0x7d, 0x10, 0x3a, 0x8d, 0xeb, 0xb0, 0xc5, 0x3f, 0x14, 0x00, 0x51, + 0x45, 0xd5, 0xd9, 0x9e, 0xfe, 0xd7, 0x65, 0x3c, 0xdd, 0x85, 0x22, 0x8d, 0x7c, 0x94, 0xf5, 0x3d, + 0xb9, 0xa3, 0xdb, 0xb6, 0x6e, 0xb4, 0xf8, 0x80, 0xca, 0x53, 0x7a, 0x7d, 0x6f, 0x93, 0x51, 0xc5, + 0xbf, 0x0c, 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, 0x3d, 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, 0xeb, 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, 0xc3, 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, 0x51, 0x80, 0x0b, 0x5f, 0xd9, 0x56, 0xec, 0xff, + 0x0f, 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, 0xf7, 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, 0xc9, 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, + 0x34, 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, 0x22, 0x2e, 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, 0xa3, 0x5d, 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, 0x41, 0xc7, 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, 0x37, 0x13, 0x23, + 0x8f, 0x6e, 0x7a, 0x47, 0x09, 0x14, 0x47, 0xa1, 0xc7, 0x37, 0x11, 0xa4, 0xc8, 0x6f, 0x76, 0xca, + 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, 0xcf, 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, 0x2e, 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, 0xa3, 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, 0x5a, + 0x9b, 0xc5, 0xf0, 0xd3, 0x36, 0x04, 0x8e, 0x60, 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, 0xcf, 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, 0x2f, 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, 0xf7, 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, 0x88, 0xc4, 0x9d, 0xad, 0xef, 0x45, + 0xc8, 0x8b, 0xaa, 0x08, 0x8b, 0x15, 0x74, 0x03, 0xd7, 0xfb, 0x92, 0x92, 0xd8, 0xa8, 0x02, 0xd9, + 0x60, 0x7e, 0x93, 0xe4, 0xc4, 0xf9, 0x4d, 0xc0, 0xf2, 0xb2, 0x9a, 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, 0xcb, 0xf3, 0xf4, 0xa7, 0xdb, 0x50, 0x4c, 0xb3, 0xf0, 0x84, 0xfa, 0x9a, + 0x94, 0x66, 0x92, 0xeb, 0x9a, 0xf8, 0x9f, 0x04, 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, 0xfe, 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, 0x3c, 0x92, 0x67, 0xf4, 0x15, + 0x9b, 0x1d, 0xb9, 0x61, 0x87, 0x09, 0x7c, 0xd5, 0xc4, 0x09, 0x89, 0xff, 0x9e, 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, 0x26, 0x40, 0xbe, 0xd1, 0xdb, 0x65, 0xf9, 0xa5, 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, 0xe3, 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, 0x4e, 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, 0x3f, 0x94, 0x61, 0x8e, 0xd7, 0x7b, 0xc7, 0xd0, 0x4d, 0x03, 0x3d, 0x80, 0x64, + 0x8b, 0xbf, 0xe5, 0xc8, 0x46, 0xee, 0x48, 0xfa, 0x79, 0xec, 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, 0x99, 0xd1, 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, + 0x99, 0x29, 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, 0xaf, 0x0a, 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, 0x4f, 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, 0x31, 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, 0xa7, 0xba, 0xa6, 0x27, + 0xff, 0x54, 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, 0xc1, 0x42, 0x1d, 0xe5, 0x98, + 0x7e, 0xf7, 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, 0x12, 0x2a, 0xd0, 0xa6, 0xf2, 0x2f, 0x54, 0xb0, 0x6f, + 0x09, 0x3d, 0x86, 0x32, 0xcf, 0xe2, 0x7f, 0xa4, 0x58, 0x9a, 0x4c, 0xb5, 0xee, 0x8f, 0xcf, 0x22, + 0x15, 0x7b, 0x99, 0x65, 0xed, 0x27, 0x05, 0x88, 0xaa, 0xfd, 0x39, 0xb4, 0x05, 0xf3, 0x6a, 0x5b, + 0xf7, 0x10, 0x04, 0x6b, 0xf9, 0xfc, 0xd0, 0xd9, 0x54, 0xa1, 0x65, 0xfd, 0x97, 0xd1, 0x05, 0x35, + 0x4c, 0x40, 0x0d, 0xa0, 0x79, 0x75, 0xe4, 0xae, 0xd9, 0xd6, 0xd5, 0x13, 0x0a, 0x1e, 0xc2, 0xd9, + 0xc8, 0x47, 0x7e, 0x13, 0xe0, 0x95, 0xa2, 0x3b, 0xdb, 0x94, 0x53, 0x82, 0x23, 0xef, 0x37, 0xcb, + 0x24, 0xbc, 0x9e, 0x4a, 0xcf, 0x15, 0x73, 0xeb, 0xa9, 0x74, 0xbe, 0x58, 0x10, 0xff, 0xa6, 0x00, + 0x85, 0xbe, 0xba, 0xa0, 0x1a, 0x5c, 0xd4, 0xbc, 0xa9, 0x22, 0xf3, 0x43, 0x5d, 0xba, 0x69, 0xf0, + 0x04, 0xeb, 0x0b, 0x5f, 0x9e, 0x2e, 0x16, 0x68, 0xe9, 0xa7, 0xde, 0x2d, 0xe9, 0x82, 0xcf, 0xe1, + 0x53, 0xd1, 0x47, 0x90, 0x67, 0xee, 0xa7, 0xf7, 0x6d, 0x3c, 0x1a, 0x8d, 0xbf, 0x3a, 0xff, 0xe5, + 0xe9, 0x62, 0x8e, 0xfa, 0x9c, 0x6e, 0x02, 0x64, 0x29, 0xd7, 0x0e, 0x5e, 0x8a, 0xbf, 0x2e, 0xc0, + 0x5c, 0xe8, 0xe0, 0xd4, 0xe3, 0xbe, 0x97, 0xdf, 0x57, 0xa2, 0x71, 0xeb, 0xb0, 0xd0, 0xc5, 0x34, + 0x1f, 0xe7, 0x6e, 0x1c, 0xe8, 0xe2, 0x70, 0xdc, 0x43, 0x77, 0x71, 0xdc, 0xa0, 0x18, 0x97, 0xed, + 0xe3, 0xd4, 0x0f, 0x7e, 0xb8, 0x38, 0x25, 0xfe, 0x7e, 0x0a, 0x72, 0xe1, 0x63, 0x52, 0xf5, 0xbe, + 0x7a, 0x45, 0xad, 0x2b, 0x21, 0x8e, 0xa5, 0x11, 0x09, 0x22, 0x33, 0xfe, 0x97, 0x0c, 0x58, 0x35, + 0xaf, 0x8d, 0x78, 0xc5, 0x1f, 0xac, 0xa7, 0xcf, 0x58, 0xfe, 0x5e, 0xd2, 0xb3, 0xaf, 0x4b, 0x30, + 0x4d, 0x13, 0x16, 0xf1, 0xaa, 0x95, 0xfa, 0x47, 0x0f, 0xf1, 0xb5, 0xc9, 0x7d, 0x89, 0x15, 0x23, + 0xf6, 0xb8, 0xf9, 0x46, 0x19, 0x01, 0xfd, 0x69, 0x70, 0xfe, 0x4f, 0x11, 0xf2, 0x8c, 0x90, 0xd3, + 0xe7, 0xcb, 0x08, 0x89, 0x7e, 0x09, 0x0a, 0xaa, 0xd9, 0x6e, 0xb3, 0xb5, 0x92, 0x59, 0xa4, 0xc1, + 0x1c, 0x2f, 0x54, 0x04, 0xff, 0x7a, 0xe4, 0x92, 0xf7, 0x15, 0xc9, 0x25, 0x89, 0x7f, 0x45, 0x32, + 0x10, 0x55, 0x9b, 0xf7, 0x84, 0x31, 0x43, 0xd6, 0x17, 0xe0, 0x3b, 0xfb, 0x26, 0x01, 0xbe, 0x2c, + 0x28, 0x9c, 0x8f, 0x9c, 0x3f, 0x11, 0x78, 0x08, 0xce, 0x33, 0xd3, 0x3c, 0xe8, 0x79, 0x81, 0xb9, + 0xe5, 0x60, 0x7e, 0xc6, 0xf4, 0x97, 0xa7, 0x8b, 0x29, 0xc9, 0x4b, 0xd0, 0x18, 0x65, 0xf9, 0x13, + 0x3f, 0x9d, 0xe5, 0xbf, 0x0e, 0x73, 0x5d, 0x0b, 0xef, 0x61, 0x47, 0xdd, 0x97, 0x8d, 0x5e, 0x87, + 0x9f, 0xdf, 0xc9, 0xba, 0xb4, 0xad, 0x5e, 0x07, 0xdd, 0x83, 0xa2, 0x57, 0x84, 0x63, 0x74, 0x37, + 0x81, 0x97, 0x4b, 0xe7, 0x88, 0x5e, 0xfc, 0xdf, 0x02, 0x2c, 0x84, 0xda, 0xc4, 0xe7, 0xc4, 0x3a, + 0x64, 0x7d, 0x73, 0x60, 0x97, 0x84, 0x73, 0x86, 0xa9, 0x06, 0x99, 0x91, 0x0c, 0x97, 0xdc, 0xc7, + 0xd2, 0xec, 0xfd, 0xbe, 0xd8, 0xc4, 0x39, 0xc5, 0x5e, 0xf4, 0xe5, 0xac, 0x05, 0x1e, 0xe0, 0x4d, + 0x92, 0xe4, 0x44, 0x93, 0x44, 0xfc, 0x2d, 0x01, 0x8a, 0xf4, 0x01, 0x4f, 0x30, 0xd6, 0x62, 0xb1, + 0x4e, 0x6e, 0xf8, 0x77, 0x62, 0xf2, 0x13, 0x3a, 0xa1, 0x2f, 0x8e, 0x24, 0xc3, 0x5f, 0x1c, 0x11, + 0x7f, 0x28, 0x40, 0xde, 0xab, 0x21, 0xfb, 0x1e, 0xdf, 0x88, 0x34, 0xa0, 0x6f, 0xf6, 0xcd, 0x39, + 0x37, 0xdf, 0xc9, 0x44, 0x9f, 0x08, 0x0c, 0xe6, 0x3b, 0x61, 0xdf, 0x4a, 0xfb, 0xdb, 0xee, 0xc8, + 0x21, 0x55, 0xac, 0xf8, 0x79, 0x2d, 0xde, 0xe0, 0xb0, 0x92, 0x44, 0x3f, 0x65, 0x6a, 0xb6, 0x0f, + 0x59, 0x8a, 0x99, 0x89, 0xcc, 0x16, 0xe2, 0x81, 0x67, 0xc0, 0x77, 0xf3, 0xb4, 0x66, 0x83, 0x7e, + 0xe4, 0x94, 0xfd, 0xb6, 0xc5, 0x27, 0x01, 0x05, 0xd2, 0xce, 0x27, 0x5a, 0x9a, 0xc8, 0x94, 0xba, + 0x5a, 0x62, 0x63, 0xe5, 0x8f, 0x83, 0x3d, 0x51, 0x3d, 0x24, 0x28, 0xee, 0x11, 0x24, 0x0f, 0x95, + 0xf6, 0xa8, 0x08, 0xae, 0x50, 0xcf, 0x49, 0xa4, 0x34, 0x7a, 0x12, 0x4a, 0x07, 0x92, 0x18, 0x8e, + 0x38, 0x06, 0x55, 0x1a, 0x4a, 0x1b, 0xf2, 0x61, 0x78, 0xac, 0x8f, 0x7c, 0x7c, 0x70, 0xd0, 0x7f, + 0x9c, 0xfa, 0xd1, 0x0f, 0x17, 0x05, 0xf1, 0x13, 0x40, 0x12, 0xb6, 0xb1, 0xf3, 0xa2, 0x67, 0x5a, + 0x7e, 0x6a, 0x95, 0xdb, 0x7d, 0x9f, 0x60, 0x99, 0x5e, 0xcd, 0x9e, 0x45, 0x25, 0x4f, 0xba, 0x08, + 0x0b, 0x21, 0x6e, 0x66, 0x2c, 0xc4, 0x0f, 0xe1, 0xca, 0x53, 0xd3, 0xb6, 0xf5, 0x2e, 0x81, 0xae, + 0x74, 0x56, 0x92, 0xa5, 0xc1, 0x33, 0x8f, 0xe9, 0x2e, 0xdd, 0xc4, 0x30, 0x98, 0x19, 0xc9, 0x48, + 0xde, 0xb5, 0xf8, 0x07, 0x02, 0x5c, 0x1e, 0xe4, 0x64, 0x5a, 0x8e, 0x3a, 0x5b, 0x39, 0xab, 0x9a, + 0x7e, 0x26, 0xbd, 0xf1, 0xa3, 0xd5, 0x2d, 0x4e, 0x5c, 0x4c, 0xfe, 0x4c, 0xb9, 0xa3, 0x50, 0xf3, + 0xc1, 0xcf, 0x79, 0xe7, 0x39, 0x79, 0x93, 0x51, 0x7d, 0x4b, 0x92, 0x9a, 0xcc, 0x92, 0x34, 0xa1, + 0xb0, 0x6e, 0xea, 0x06, 0xf1, 0x64, 0xdd, 0xf6, 0xae, 0x40, 0x7e, 0x57, 0x37, 0x14, 0xeb, 0x44, + 0x76, 0x03, 0x07, 0x85, 0x71, 0x81, 0x83, 0x52, 0x8e, 0x71, 0xf0, 0x4b, 0xf1, 0xc7, 0x02, 0x14, + 0x7d, 0xb1, 0xdc, 0x22, 0xbf, 0x0b, 0xa0, 0xb6, 0x7b, 0xb6, 0x83, 0x2d, 0xb7, 0x97, 0xe6, 0x58, + 0x9c, 0x7c, 0x85, 0x51, 0xeb, 0x6b, 0x52, 0x86, 0x17, 0xa8, 0x6b, 0xe8, 0x46, 0x38, 0x0d, 0xc5, + 0xf4, 0x2a, 0x9c, 0x0d, 0x24, 0x9f, 0x20, 0xdd, 0x6e, 0x3b, 0xa6, 0xe5, 0x61, 0x26, 0xde, 0xed, + 0x6e, 0xd2, 0x7a, 0x7a, 0x7a, 0x9a, 0x94, 0x5b, 0x81, 0x3c, 0x59, 0xee, 0x0f, 0xb1, 0xd7, 0xa4, + 0xd4, 0xf8, 0x26, 0x31, 0x0e, 0xb7, 0x49, 0xbf, 0x4b, 0x9c, 0x54, 0xd6, 0x1b, 0x5e, 0x0f, 0x8f, + 0xb0, 0x68, 0x3f, 0x13, 0x4c, 0xfe, 0x38, 0x59, 0x7a, 0x4c, 0xea, 0x89, 0x7c, 0x0b, 0xd2, 0xee, + 0x47, 0xa2, 0xf9, 0x04, 0xb9, 0xb2, 0xc4, 0xbe, 0x22, 0xbd, 0xe4, 0x7e, 0x45, 0x7a, 0x69, 0x8d, + 0x17, 0x60, 0x66, 0xfc, 0x07, 0xff, 0x75, 0x51, 0x90, 0x3c, 0xa6, 0xfb, 0x0d, 0x32, 0xc2, 0x07, + 0x56, 0x61, 0x94, 0x07, 0x08, 0x7c, 0xad, 0x87, 0x7f, 0x37, 0x78, 0x65, 0x4d, 0xde, 0xd9, 0xaa, + 0x3c, 0xdf, 0xdc, 0xac, 0x37, 0x9b, 0xd5, 0xb5, 0xa2, 0x80, 0x8a, 0x30, 0x17, 0xfa, 0xd6, 0x4f, + 0x82, 0x7d, 0x49, 0xf8, 0xfe, 0xcf, 0x00, 0xf8, 0x9f, 0x0d, 0x23, 0xb2, 0x36, 0xaa, 0x9f, 0xc9, + 0x2f, 0x57, 0x9e, 0xed, 0x54, 0x1b, 0xc5, 0x29, 0x84, 0x20, 0xbf, 0xba, 0xd2, 0xac, 0xd4, 0x64, + 0xa9, 0xda, 0xd8, 0x7e, 0xbe, 0xd5, 0xa8, 0xba, 0x5f, 0x20, 0xbe, 0xbf, 0x06, 0x73, 0xc1, 0xc4, + 0x42, 0x68, 0x01, 0x0a, 0x95, 0x5a, 0xb5, 0xb2, 0x21, 0xbf, 0xac, 0xaf, 0xc8, 0x2f, 0x76, 0xaa, + 0x3b, 0xd5, 0xe2, 0x14, 0xad, 0x1a, 0x25, 0x3e, 0xd9, 0x79, 0xf6, 0xac, 0x28, 0xa0, 0x02, 0x64, + 0xd9, 0x35, 0xfd, 0x2e, 0x50, 0x31, 0x71, 0x7f, 0x13, 0xb2, 0x81, 0x04, 0xc2, 0xe4, 0x71, 0xdb, + 0x3b, 0x8d, 0x9a, 0xdc, 0xac, 0x6f, 0x56, 0x1b, 0xcd, 0x95, 0xcd, 0x6d, 0x26, 0x83, 0xd2, 0x56, + 0x56, 0x9f, 0x4b, 0xcd, 0xa2, 0xe0, 0x5d, 0x37, 0x9f, 0xef, 0x54, 0x6a, 0x6e, 0x33, 0xc4, 0x54, + 0x3a, 0x59, 0x4c, 0xde, 0xff, 0x2b, 0x02, 0x5c, 0x1e, 0x92, 0x64, 0x07, 0x65, 0x61, 0x76, 0xc7, + 0xa0, 0xd9, 0x54, 0x8b, 0x53, 0x28, 0x17, 0xc8, 0xb3, 0x53, 0x14, 0x50, 0x9a, 0x65, 0x3a, 0x29, + 0x26, 0xd0, 0x0c, 0x24, 0x1a, 0x8f, 0x8a, 0x49, 0x52, 0xd3, 0x40, 0x9a, 0x9a, 0x62, 0x0a, 0x65, + 0x78, 0x82, 0x8c, 0xe2, 0x34, 0x9a, 0xf3, 0xf3, 0x54, 0x14, 0x67, 0x88, 0x28, 0x2f, 0xdf, 0x43, + 0x71, 0xf6, 0xfe, 0x75, 0x08, 0x9c, 0xa9, 0x47, 0x00, 0x33, 0xcf, 0x14, 0x07, 0xdb, 0x4e, 0x71, + 0x0a, 0xcd, 0x42, 0x72, 0xa5, 0xdd, 0x2e, 0x0a, 0x0f, 0xff, 0x55, 0x0a, 0xd2, 0xee, 0xf7, 0x6f, + 0xd0, 0x33, 0x98, 0x66, 0x5b, 0xd7, 0x8b, 0xc3, 0x3d, 0x7b, 0x3a, 0x79, 0xcb, 0xd7, 0xc6, 0xb9, + 0xfe, 0xe2, 0x14, 0xfa, 0x4b, 0x90, 0x0d, 0x78, 0x4c, 0x68, 0xe8, 0xf6, 0x5b, 0xc8, 0x4b, 0x2c, + 0xdf, 0x1e, 0x57, 0xcc, 0x93, 0xff, 0x0a, 0x32, 0x9e, 0x05, 0x47, 0x37, 0x46, 0xd9, 0x77, 0x57, + 0xf6, 0xe8, 0x45, 0x80, 0xcc, 0x35, 0x71, 0xea, 0x7d, 0x01, 0x59, 0x80, 0x06, 0x8d, 0x2d, 0x8a, + 0x0a, 0x88, 0x18, 0x6a, 0xcd, 0xcb, 0xf7, 0x27, 0x2a, 0xed, 0x3f, 0x93, 0x28, 0xcb, 0x5f, 0x31, + 0xa2, 0x95, 0x35, 0xb0, 0x1e, 0x45, 0x2b, 0x2b, 0x62, 0xe1, 0x99, 0x42, 0x2f, 0x20, 0x45, 0x2c, + 0x25, 0x8a, 0xf2, 0x21, 0xfb, 0x2c, 0x73, 0xf9, 0xc6, 0xc8, 0x32, 0xae, 0xc8, 0xd5, 0x7b, 0x3f, + 0xfa, 0xf3, 0xab, 0x53, 0x3f, 0x3a, 0xbb, 0x2a, 0xfc, 0xf8, 0xec, 0xaa, 0xf0, 0xa7, 0x67, 0x57, + 0x85, 0x3f, 0x3b, 0xbb, 0x2a, 0x7c, 0xff, 0x27, 0x57, 0xa7, 0x7e, 0xfc, 0x93, 0xab, 0x53, 0x7f, + 0xfa, 0x93, 0xab, 0x53, 0x9f, 0xcf, 0x72, 0xee, 0xdd, 0x19, 0x6a, 0x54, 0x1e, 0xfd, 0xbf, 0x00, + 0x00, 0x00, 0xff, 0xff, 0x10, 0xb4, 0xda, 0x0c, 0xd5, 0x7f, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index f09a518503d2..1a4b2a727573 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1673,6 +1673,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. // @@ -1724,6 +1744,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; } @@ -1775,6 +1796,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/server/init.go b/pkg/server/init.go index e1796b8d2616..ce1741f7fea5 100644 --- a/pkg/server/init.go +++ b/pkg/server/init.go @@ -16,6 +16,7 @@ import ( "sync" "time" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/gossip/resolver" @@ -559,6 +560,9 @@ type initServerCfg struct { // n2 is `cockroach init`-ialized, n3 will learn about it. The reverse will // not be true. resolvers []resolver.Resolver + + // testingKnobs is used for internal test controls only. + testingKnobs base.TestingKnobs } func newInitServerConfig(cfg Config, dialOpts []grpc.DialOption) initServerCfg { @@ -578,5 +582,6 @@ func newInitServerConfig(cfg Config, dialOpts []grpc.DialOption) initServerCfg { defaultZoneConfig: cfg.DefaultZoneConfig, dialOpts: dialOpts, resolvers: resolvers, + testingKnobs: cfg.TestingKnobs, } } diff --git a/pkg/server/migration.go b/pkg/server/migration.go index 41b058f09201..1e66bc3c50aa 100644 --- a/pkg/server/migration.go +++ b/pkg/server/migration.go @@ -143,3 +143,49 @@ func (m *migrationServer) BumpClusterVersion( resp := &serverpb.BumpClusterVersionResponse{} return resp, nil } + +// SyncAllEngines implements the MigrationServer interface. +func (m *migrationServer) SyncAllEngines( + ctx context.Context, _ *serverpb.SyncAllEnginesRequest, +) (*serverpb.SyncAllEnginesResponse, error) { + ctx, span := m.server.AnnotateCtxWithSpan(ctx, "sync-all-engines") + defer span.Finish() + ctx = logtags.AddTag(ctx, "sync-all-engines", nil) + + // Let's be paranoid here and ensure that all stores have been fully + // initialized. + m.server.node.waitForAdditionalStoreInit() + + for _, eng := range m.server.engines { + batch := eng.NewBatch() + if err := batch.LogData(nil); err != nil { + return nil, err + } + } + + log.Infof(ctx, "synced %d engines", len(m.server.engines)) + resp := &serverpb.SyncAllEnginesResponse{} + return resp, nil +} + +// PurgeOutdatedReplicas implements the MigrationServer interface. +func (m *migrationServer) PurgeOutdatedReplicas( + ctx context.Context, req *serverpb.PurgeOutdatedReplicasRequest, +) (*serverpb.PurgeOutdatedReplicasResponse, error) { + ctx, span := m.server.AnnotateCtxWithSpan(ctx, "purged-outdated-replicas") + defer span.Finish() + ctx = logtags.AddTag(ctx, "purge-outdated-replicas", nil) + + // Same as in SyncAllEngines, because stores can be added asynchronously, we + // need to ensure that the bootstrap process has happened. + m.server.node.waitForAdditionalStoreInit() + + if err := m.server.node.stores.VisitStores(func(s *kvserver.Store) error { + return s.PurgeOutdatedReplicas(ctx, *req.Version) + }); err != nil { + return nil, err + } + + resp := &serverpb.PurgeOutdatedReplicasResponse{} + return resp, nil +} diff --git a/pkg/server/migration_test.go b/pkg/server/migration_test.go index 5d20624ac321..42b79fad3703 100644 --- a/pkg/server/migration_test.go +++ b/pkg/server/migration_test.go @@ -197,3 +197,37 @@ func TestBumpClusterVersion(t *testing.T) { }) } } + +func TestMigrationPurgeOutdatedReplicas(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{ + PurgeOutdatedReplicasInterceptor: func() { + intercepted++ + }, + }, + }, + }) + defer s.Stopper().Stop(context.Background()) + + migrationServer := s.MigrationServer().(*migrationServer) + if _, err := migrationServer.PurgeOutdatedReplicas(context.Background(), &serverpb.PurgeOutdatedReplicasRequest{ + Version: &clusterversion.TestingBinaryVersion, + }); err != nil { + t.Fatal(err) + } + + if intercepted != numStores { + t.Fatalf("expected to have GC-ed replicas on %d stores, found %d", numStores, intercepted) + } +} diff --git a/pkg/server/node.go b/pkg/server/node.go index e5ef699a4cdb..dee1afd1e01c 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -258,10 +258,14 @@ func bootstrapCluster( return splits[i].Less(splits[j]) }) + var storeKnobs kvserver.StoreTestingKnobs + if kn, ok := initCfg.testingKnobs.Store.(*kvserver.StoreTestingKnobs); ok { + storeKnobs = *kn + } if err := kvserver.WriteInitialClusterData( ctx, eng, initialValues, bootstrapVersion.Version, len(engines), splits, - hlc.UnixNano(), + hlc.UnixNano(), storeKnobs, ); err != nil { return nil, err } diff --git a/pkg/server/serverpb/migration.pb.go b/pkg/server/serverpb/migration.pb.go index e2eba51c12f3..de77f398509b 100644 --- a/pkg/server/serverpb/migration.pb.go +++ b/pkg/server/serverpb/migration.pb.go @@ -7,6 +7,7 @@ import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" import clusterversion "github.com/cockroachdb/cockroach/pkg/clusterversion" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import ( context "context" @@ -36,7 +37,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_878be5ae941b79ed, []int{0} } func (m *ValidateTargetClusterVersionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -70,7 +71,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_878be5ae941b79ed, []int{1} } func (m *ValidateTargetClusterVersionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -95,8 +96,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 +106,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_878be5ae941b79ed, []int{2} } func (m *BumpClusterVersionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -138,7 +139,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_878be5ae941b79ed, []int{3} } func (m *BumpClusterVersionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -163,11 +164,151 @@ func (m *BumpClusterVersionResponse) XXX_DiscardUnknown() { var xxx_messageInfo_BumpClusterVersionResponse proto.InternalMessageInfo +// PurgeOutdatedReplicasRequest is used to instruct the target node to +// purge all replicas with a version less than the one provided. +type PurgeOutdatedReplicasRequest struct { + Version *roachpb.Version `protobuf:"bytes,1,opt,name=version,proto3" json:"version,omitempty"` +} + +func (m *PurgeOutdatedReplicasRequest) Reset() { *m = PurgeOutdatedReplicasRequest{} } +func (m *PurgeOutdatedReplicasRequest) String() string { return proto.CompactTextString(m) } +func (*PurgeOutdatedReplicasRequest) ProtoMessage() {} +func (*PurgeOutdatedReplicasRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_migration_878be5ae941b79ed, []int{4} +} +func (m *PurgeOutdatedReplicasRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PurgeOutdatedReplicasRequest) 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 *PurgeOutdatedReplicasRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_PurgeOutdatedReplicasRequest.Merge(dst, src) +} +func (m *PurgeOutdatedReplicasRequest) XXX_Size() int { + return m.Size() +} +func (m *PurgeOutdatedReplicasRequest) XXX_DiscardUnknown() { + xxx_messageInfo_PurgeOutdatedReplicasRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_PurgeOutdatedReplicasRequest proto.InternalMessageInfo + +// PurgeOutdatedReplicasResponse is the response to a +// PurgeOutdatedReplicasRequest. +type PurgeOutdatedReplicasResponse struct { +} + +func (m *PurgeOutdatedReplicasResponse) Reset() { *m = PurgeOutdatedReplicasResponse{} } +func (m *PurgeOutdatedReplicasResponse) String() string { return proto.CompactTextString(m) } +func (*PurgeOutdatedReplicasResponse) ProtoMessage() {} +func (*PurgeOutdatedReplicasResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_migration_878be5ae941b79ed, []int{5} +} +func (m *PurgeOutdatedReplicasResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PurgeOutdatedReplicasResponse) 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 *PurgeOutdatedReplicasResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_PurgeOutdatedReplicasResponse.Merge(dst, src) +} +func (m *PurgeOutdatedReplicasResponse) XXX_Size() int { + return m.Size() +} +func (m *PurgeOutdatedReplicasResponse) XXX_DiscardUnknown() { + xxx_messageInfo_PurgeOutdatedReplicasResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_PurgeOutdatedReplicasResponse proto.InternalMessageInfo + +// SyncAllEnginesRequest is used to instruct the target node to sync all its +// engines. +type SyncAllEnginesRequest struct { +} + +func (m *SyncAllEnginesRequest) Reset() { *m = SyncAllEnginesRequest{} } +func (m *SyncAllEnginesRequest) String() string { return proto.CompactTextString(m) } +func (*SyncAllEnginesRequest) ProtoMessage() {} +func (*SyncAllEnginesRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_migration_878be5ae941b79ed, []int{6} +} +func (m *SyncAllEnginesRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SyncAllEnginesRequest) 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 *SyncAllEnginesRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SyncAllEnginesRequest.Merge(dst, src) +} +func (m *SyncAllEnginesRequest) XXX_Size() int { + return m.Size() +} +func (m *SyncAllEnginesRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SyncAllEnginesRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_SyncAllEnginesRequest proto.InternalMessageInfo + +// SyncAllEnginesResponse is the response to a SyncAllEnginesRequest. +type SyncAllEnginesResponse struct { +} + +func (m *SyncAllEnginesResponse) Reset() { *m = SyncAllEnginesResponse{} } +func (m *SyncAllEnginesResponse) String() string { return proto.CompactTextString(m) } +func (*SyncAllEnginesResponse) ProtoMessage() {} +func (*SyncAllEnginesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_migration_878be5ae941b79ed, []int{7} +} +func (m *SyncAllEnginesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SyncAllEnginesResponse) 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 *SyncAllEnginesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SyncAllEnginesResponse.Merge(dst, src) +} +func (m *SyncAllEnginesResponse) XXX_Size() int { + return m.Size() +} +func (m *SyncAllEnginesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SyncAllEnginesResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_SyncAllEnginesResponse 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((*PurgeOutdatedReplicasRequest)(nil), "cockroach.server.serverpb.PurgeOutdatedReplicasRequest") + proto.RegisterType((*PurgeOutdatedReplicasResponse)(nil), "cockroach.server.serverpb.PurgeOutdatedReplicasResponse") + proto.RegisterType((*SyncAllEnginesRequest)(nil), "cockroach.server.serverpb.SyncAllEnginesRequest") + proto.RegisterType((*SyncAllEnginesResponse)(nil), "cockroach.server.serverpb.SyncAllEnginesResponse") } // Reference imports to suppress errors if they are not otherwise used. @@ -196,6 +337,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) + // SyncAllEngines is used to instruct the target node to sync all its + // engines. + SyncAllEngines(ctx context.Context, in *SyncAllEnginesRequest, opts ...grpc.CallOption) (*SyncAllEnginesResponse, error) + // PurgeOutdatedReplicas is used to instruct the target node to purge all + // replicas with a version less than the one provided. + PurgeOutdatedReplicas(ctx context.Context, in *PurgeOutdatedReplicasRequest, opts ...grpc.CallOption) (*PurgeOutdatedReplicasResponse, error) } type migrationClient struct { @@ -224,6 +371,24 @@ func (c *migrationClient) BumpClusterVersion(ctx context.Context, in *BumpCluste return out, nil } +func (c *migrationClient) SyncAllEngines(ctx context.Context, in *SyncAllEnginesRequest, opts ...grpc.CallOption) (*SyncAllEnginesResponse, error) { + out := new(SyncAllEnginesResponse) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Migration/SyncAllEngines", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *migrationClient) PurgeOutdatedReplicas(ctx context.Context, in *PurgeOutdatedReplicasRequest, opts ...grpc.CallOption) (*PurgeOutdatedReplicasResponse, error) { + out := new(PurgeOutdatedReplicasResponse) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Migration/PurgeOutdatedReplicas", 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 +405,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) + // SyncAllEngines is used to instruct the target node to sync all its + // engines. + SyncAllEngines(context.Context, *SyncAllEnginesRequest) (*SyncAllEnginesResponse, error) + // PurgeOutdatedReplicas is used to instruct the target node to purge all + // replicas with a version less than the one provided. + PurgeOutdatedReplicas(context.Context, *PurgeOutdatedReplicasRequest) (*PurgeOutdatedReplicasResponse, error) } func RegisterMigrationServer(s *grpc.Server, srv MigrationServer) { @@ -282,6 +453,42 @@ func _Migration_BumpClusterVersion_Handler(srv interface{}, ctx context.Context, return interceptor(ctx, in, info, handler) } +func _Migration_SyncAllEngines_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(SyncAllEnginesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(MigrationServer).SyncAllEngines(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/cockroach.server.serverpb.Migration/SyncAllEngines", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(MigrationServer).SyncAllEngines(ctx, req.(*SyncAllEnginesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Migration_PurgeOutdatedReplicas_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(PurgeOutdatedReplicasRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(MigrationServer).PurgeOutdatedReplicas(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/cockroach.server.serverpb.Migration/PurgeOutdatedReplicas", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(MigrationServer).PurgeOutdatedReplicas(ctx, req.(*PurgeOutdatedReplicasRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _Migration_serviceDesc = grpc.ServiceDesc{ ServiceName: "cockroach.server.serverpb.Migration", HandlerType: (*MigrationServer)(nil), @@ -294,6 +501,14 @@ var _Migration_serviceDesc = grpc.ServiceDesc{ MethodName: "BumpClusterVersion", Handler: _Migration_BumpClusterVersion_Handler, }, + { + MethodName: "SyncAllEngines", + Handler: _Migration_SyncAllEngines_Handler, + }, + { + MethodName: "PurgeOutdatedReplicas", + Handler: _Migration_PurgeOutdatedReplicas_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "server/serverpb/migration.proto", @@ -391,6 +606,88 @@ func (m *BumpClusterVersionResponse) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *PurgeOutdatedReplicasRequest) 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 *PurgeOutdatedReplicasRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Version != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintMigration(dAtA, i, uint64(m.Version.Size())) + n3, err := m.Version.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n3 + } + return i, nil +} + +func (m *PurgeOutdatedReplicasResponse) 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 *PurgeOutdatedReplicasResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + +func (m *SyncAllEnginesRequest) 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 *SyncAllEnginesRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + +func (m *SyncAllEnginesResponse) 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 *SyncAllEnginesResponse) 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 +741,46 @@ func (m *BumpClusterVersionResponse) Size() (n int) { return n } +func (m *PurgeOutdatedReplicasRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Version != nil { + l = m.Version.Size() + n += 1 + l + sovMigration(uint64(l)) + } + return n +} + +func (m *PurgeOutdatedReplicasResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + +func (m *SyncAllEnginesRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + +func (m *SyncAllEnginesResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + func sovMigration(x uint64) (n int) { for { n++ @@ -723,6 +1060,239 @@ func (m *BumpClusterVersionResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *PurgeOutdatedReplicasRequest) 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: PurgeOutdatedReplicasRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PurgeOutdatedReplicasRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Version", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMigration + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthMigration + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Version == nil { + m.Version = &roachpb.Version{} + } + if err := m.Version.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + 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 *PurgeOutdatedReplicasResponse) 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: PurgeOutdatedReplicasResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: PurgeOutdatedReplicasResponse: 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 *SyncAllEnginesRequest) 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: SyncAllEnginesRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SyncAllEnginesRequest: 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 *SyncAllEnginesResponse) 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: SyncAllEnginesResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SyncAllEnginesResponse: 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,27 +1399,35 @@ var ( ) func init() { - proto.RegisterFile("server/serverpb/migration.proto", fileDescriptor_migration_8dfeb6fcf9144e4c) + proto.RegisterFile("server/serverpb/migration.proto", fileDescriptor_migration_878be5ae941b79ed) } -var fileDescriptor_migration_8dfeb6fcf9144e4c = []byte{ - // 278 bytes of a gzipped FileDescriptorProto +var fileDescriptor_migration_878be5ae941b79ed = []byte{ + // 409 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, 0xc9, 0x19, 0x7a, 0x10, 0x35, 0x7a, 0x30, 0xa5, 0x52, 0x2a, 0xc9, 0x39, 0xa5, 0xc5, 0x25, 0x60, - 0x5e, 0x71, 0x66, 0x7e, 0x9e, 0x3e, 0x94, 0x1b, 0x0f, 0xe5, 0x43, 0x0c, 0x50, 0xaa, 0xe0, 0x52, - 0x0e, 0x4b, 0xcc, 0xc9, 0x4c, 0x49, 0x2c, 0x49, 0x0d, 0x49, 0x2c, 0x4a, 0x4f, 0x2d, 0x71, 0x86, - 0x28, 0x0b, 0x83, 0xa8, 0x0a, 0x4a, 0x2d, 0x2c, 0x4d, 0x2d, 0x2e, 0x11, 0x0a, 0xe4, 0xe2, 0x47, - 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, + 0x5e, 0x71, 0x66, 0x7e, 0x9e, 0x3e, 0x94, 0x1b, 0x0f, 0xe5, 0x43, 0x0c, 0x90, 0x12, 0x03, 0x6b, + 0x06, 0x99, 0x9c, 0x5a, 0x92, 0x98, 0x92, 0x58, 0x92, 0x08, 0x11, 0x57, 0xaa, 0xe0, 0x52, 0x0e, + 0x4b, 0xcc, 0xc9, 0x4c, 0x49, 0x2c, 0x49, 0x0d, 0x49, 0x2c, 0x4a, 0x4f, 0x2d, 0x71, 0x86, 0x68, + 0x0f, 0x83, 0xe8, 0x0e, 0x4a, 0x2d, 0x2c, 0x4d, 0x2d, 0x2e, 0x11, 0x0a, 0xe4, 0xe2, 0x47, 0x33, + 0x57, 0x82, 0x51, 0x81, 0x51, 0x83, 0xdb, 0x48, 0x43, 0x0f, 0xe1, 0x32, 0x54, 0x87, 0xe8, 0xa1, + 0x99, 0xc4, 0x97, 0x8c, 0xc2, 0x57, 0x52, 0xe3, 0x52, 0xc1, 0x6f, 0x73, 0x71, 0x41, 0x7e, 0x5e, + 0x71, 0xaa, 0x52, 0x1e, 0x97, 0xa4, 0x53, 0x69, 0x6e, 0x01, 0xdd, 0xdc, 0x25, 0xc3, 0x25, 0x85, + 0xcd, 0x3e, 0xa8, 0x6b, 0x42, 0xb8, 0x64, 0x02, 0x4a, 0x8b, 0xd2, 0x53, 0xfd, 0x4b, 0x4b, 0x40, + 0x2e, 0x4f, 0x09, 0x4a, 0x2d, 0xc8, 0xc9, 0x4c, 0x4e, 0x2c, 0x86, 0x39, 0xc8, 0x84, 0x8b, 0x1d, + 0xd5, 0x21, 0x52, 0x48, 0x0e, 0x81, 0xc6, 0x81, 0x1e, 0xcc, 0x50, 0x98, 0x52, 0x25, 0x79, 0x2e, + 0x59, 0x1c, 0xa6, 0x42, 0xad, 0x15, 0xe7, 0x12, 0x0d, 0xae, 0xcc, 0x4b, 0x76, 0xcc, 0xc9, 0x71, + 0xcd, 0x4b, 0xcf, 0xcc, 0x4b, 0x85, 0xd9, 0xa7, 0x24, 0xc1, 0x25, 0x86, 0x2e, 0x01, 0xd1, 0x62, + 0xb4, 0x97, 0x85, 0x8b, 0xd3, 0x17, 0x96, 0x8c, 0x84, 0x16, 0x32, 0x72, 0xc9, 0xe0, 0x0b, 0x6e, + 0x21, 0x3b, 0x3d, 0x9c, 0x49, 0x4c, 0x8f, 0x88, 0x14, 0x22, 0x65, 0x4f, 0xb6, 0x7e, 0xa8, 0x17, + 0x19, 0x84, 0x9a, 0x19, 0xb9, 0x84, 0x30, 0x83, 0x5e, 0xc8, 0x04, 0x8f, 0xc9, 0x38, 0x53, 0x86, + 0x94, 0x29, 0x89, 0xba, 0xe0, 0xae, 0x28, 0xe7, 0xe2, 0x43, 0x0d, 0x51, 0x21, 0x03, 0x3c, 0x46, + 0x61, 0x8d, 0x15, 0x29, 0x43, 0x12, 0x74, 0xc0, 0x2d, 0xee, 0x61, 0xe4, 0x12, 0xc5, 0x9a, 0x0a, + 0x84, 0xcc, 0xf1, 0x18, 0x87, 0x2f, 0x35, 0x4a, 0x59, 0x90, 0xae, 0x11, 0xe6, 0x1c, 0x27, 0xad, + 0x13, 0x0f, 0xe5, 0x18, 0x4e, 0x3c, 0x92, 0x63, 0xbc, 0xf0, 0x48, 0x8e, 0xf1, 0xc6, 0x23, 0x39, + 0xc6, 0x07, 0x8f, 0xe4, 0x18, 0x27, 0x3c, 0x96, 0x63, 0xb8, 0xf0, 0x58, 0x8e, 0xe1, 0xc6, 0x63, + 0x39, 0x86, 0x28, 0x0e, 0x98, 0x59, 0x49, 0x6c, 0xe0, 0xc2, 0xc4, 0x18, 0x10, 0x00, 0x00, 0xff, + 0xff, 0x92, 0x08, 0x96, 0x7d, 0xc8, 0x04, 0x00, 0x00, } diff --git a/pkg/server/serverpb/migration.proto b/pkg/server/serverpb/migration.proto index 871b00dc8543..5148d36f409b 100644 --- a/pkg/server/serverpb/migration.proto +++ b/pkg/server/serverpb/migration.proto @@ -13,6 +13,7 @@ package cockroach.server.serverpb; option go_package = "serverpb"; import "clusterversion/cluster_version.proto"; +import "roachpb/metadata.proto"; // ValidateTargetClusterVersion is used to verify that the target node is // running a binary that's able to support the specified cluster version. @@ -25,8 +26,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 +35,23 @@ message BumpClusterVersionRequest { // BumpClusterVersionResponse is the response to an BumpClusterVersionRequest. message BumpClusterVersionResponse { } +// PurgeOutdatedReplicasRequest is used to instruct the target node to +// purge all replicas with a version less than the one provided. +message PurgeOutdatedReplicasRequest { + roachpb.Version version = 1; +} + +// PurgeOutdatedReplicasResponse is the response to a +// PurgeOutdatedReplicasRequest. +message PurgeOutdatedReplicasResponse{} + +// SyncAllEnginesRequest is used to instruct the target node to sync all its +// engines. +message SyncAllEnginesRequest{} + +// SyncAllEnginesResponse is the response to a SyncAllEnginesRequest. +message SyncAllEnginesResponse{} + 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 +68,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) { } + + // SyncAllEngines is used to instruct the target node to sync all its + // engines. + rpc SyncAllEngines (SyncAllEnginesRequest) returns (SyncAllEnginesResponse) { } + + // PurgeOutdatedReplicas is used to instruct the target node to purge all + // replicas with a version less than the one provided. + rpc PurgeOutdatedReplicas (PurgeOutdatedReplicasRequest) returns (PurgeOutdatedReplicasResponse) { } } diff --git a/pkg/testutils/localtestcluster/local_test_cluster.go b/pkg/testutils/localtestcluster/local_test_cluster.go index 6394bf5fd512..9362b3acc946 100644 --- a/pkg/testutils/localtestcluster/local_test_cluster.go +++ b/pkg/testutils/localtestcluster/local_test_cluster.go @@ -217,6 +217,7 @@ func (ltc *LocalTestCluster) Start(t testing.TB, baseCtx *base.Config, initFacto 1, /* numStores */ splits, ltc.Clock.PhysicalNow(), + cfg.TestingKnobs, ); err != nil { t.Fatalf("unable to start local test cluster: %s", err) } diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index ed07699a8b6d..8dca196c74ea 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",