diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index f81045162070..d08a13dca89e 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -98,6 +98,6 @@ trace.debug.enablebooleanfalseif set, traces for recent requests can be seen in the /debug page 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-4set the active cluster version in the format '.' +versionversion20.2-8set the active cluster version in the format '.' diff --git a/pkg/ccl/backupccl/system_schema.go b/pkg/ccl/backupccl/system_schema.go index f530617824a5..ce87102e2928 100644 --- a/pkg/ccl/backupccl/system_schema.go +++ b/pkg/ccl/backupccl/system_schema.go @@ -91,6 +91,9 @@ var systemTableBackupConfiguration = map[string]systemBackupConfiguration{ systemschema.DeprecatedNamespaceTable.Name: { includeInClusterBackup: optOutOfClusterBackup, }, + systemschema.MigrationsTable.Name: { + includeInClusterBackup: optOutOfClusterBackup, + }, systemschema.ProtectedTimestampsMetaTable.Name: { includeInClusterBackup: optOutOfClusterBackup, }, diff --git a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_enum b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_enum index c994ee943fea..f927cde27e79 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_enum +++ b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_enum @@ -55,4 +55,3 @@ ALTER TABLE partitioned_table_3 PARTITION BY RANGE (place) statement ok SELECT * FROM crdb_internal.tables - diff --git a/pkg/cli/testdata/doctor/testcluster b/pkg/cli/testdata/doctor/testcluster index 6d175ad4183f..2953ab3a8ff3 100644 --- a/pkg/cli/testdata/doctor/testcluster +++ b/pkg/cli/testdata/doctor/testcluster @@ -1,7 +1,7 @@ doctor cluster ---- debug doctor cluster -Examining 34 descriptors and 35 namespace entries... +Examining 35 descriptors and 36 namespace entries... Table 53: ParentID 50, ParentSchemaID 29, Name 'foo': not being dropped but no namespace entry found Examining 1 running jobs... ERROR: validation failed diff --git a/pkg/cli/testdata/zip/partial1 b/pkg/cli/testdata/zip/partial1 index 4117b05ec861..a0ac0868cc38 100644 --- a/pkg/cli/testdata/zip/partial1 +++ b/pkg/cli/testdata/zip/partial1 @@ -58,7 +58,7 @@ requesting goroutine files for node 1... writing: debug/nodes/1/goroutines.err.t ^- resulted in ... requesting log file ... requesting log file ... -requesting ranges... 35 found +requesting ranges... 36 found writing: debug/nodes/1/ranges/1.json writing: debug/nodes/1/ranges/2.json writing: debug/nodes/1/ranges/3.json @@ -94,6 +94,7 @@ writing: debug/nodes/1/ranges/32.json writing: debug/nodes/1/ranges/33.json writing: debug/nodes/1/ranges/34.json writing: debug/nodes/1/ranges/35.json +writing: debug/nodes/1/ranges/36.json writing: debug/nodes/2/status.json using SQL connection URL for node 2: postgresql://... retrieving SQL data for crdb_internal.feature_usage... writing: debug/nodes/2/crdb_internal.feature_usage.txt @@ -190,7 +191,7 @@ requesting goroutine files for node 3... writing: debug/nodes/3/goroutines.err.t ^- resulted in ... requesting log file ... requesting log file ... -requesting ranges... 35 found +requesting ranges... 36 found writing: debug/nodes/3/ranges/1.json writing: debug/nodes/3/ranges/2.json writing: debug/nodes/3/ranges/3.json @@ -226,13 +227,14 @@ writing: debug/nodes/3/ranges/32.json writing: debug/nodes/3/ranges/33.json writing: debug/nodes/3/ranges/34.json writing: debug/nodes/3/ranges/35.json +writing: debug/nodes/3/ranges/36.json requesting list of SQL databases... 3 found requesting database details for defaultdb... writing: debug/schema/defaultdb@details.json 0 tables found requesting database details for postgres... writing: debug/schema/postgres@details.json 0 tables found requesting database details for system... writing: debug/schema/system@details.json -29 tables found +30 tables found requesting table details for system.public.namespace... writing: debug/schema/system/public_namespace.json requesting table details for system.public.descriptor... writing: debug/schema/system/public_descriptor.json requesting table details for system.public.users... writing: debug/schema/system/public_users.json @@ -262,5 +264,6 @@ requesting table details for system.public.statement_diagnostics_requests... wri requesting table details for system.public.statement_diagnostics... writing: debug/schema/system/public_statement_diagnostics.json requesting table details for system.public.scheduled_jobs... writing: debug/schema/system/public_scheduled_jobs.json requesting table details for system.public.sqlliveness... writing: debug/schema/system/public_sqlliveness.json +requesting table details for system.public.migrations... writing: debug/schema/system/public_migrations.json writing: debug/pprof-summary.sh writing: debug/hot-ranges.sh diff --git a/pkg/cli/testdata/zip/partial1_excluded b/pkg/cli/testdata/zip/partial1_excluded index a01527749eb2..f1bfbd813709 100644 --- a/pkg/cli/testdata/zip/partial1_excluded +++ b/pkg/cli/testdata/zip/partial1_excluded @@ -58,7 +58,7 @@ requesting goroutine files for node 1... writing: debug/nodes/1/goroutines.err.t ^- resulted in ... requesting log file ... requesting log file ... -requesting ranges... 35 found +requesting ranges... 36 found writing: debug/nodes/1/ranges/1.json writing: debug/nodes/1/ranges/2.json writing: debug/nodes/1/ranges/3.json @@ -94,6 +94,7 @@ writing: debug/nodes/1/ranges/32.json writing: debug/nodes/1/ranges/33.json writing: debug/nodes/1/ranges/34.json writing: debug/nodes/1/ranges/35.json +writing: debug/nodes/1/ranges/36.json writing: debug/nodes/2.skipped writing: debug/nodes/3/status.json using SQL connection URL for node 3: postgresql://... @@ -124,7 +125,7 @@ requesting goroutine files for node 3... writing: debug/nodes/3/goroutines.err.t ^- resulted in ... requesting log file ... requesting log file ... -requesting ranges... 35 found +requesting ranges... 36 found writing: debug/nodes/3/ranges/1.json writing: debug/nodes/3/ranges/2.json writing: debug/nodes/3/ranges/3.json @@ -160,13 +161,14 @@ writing: debug/nodes/3/ranges/32.json writing: debug/nodes/3/ranges/33.json writing: debug/nodes/3/ranges/34.json writing: debug/nodes/3/ranges/35.json +writing: debug/nodes/3/ranges/36.json requesting list of SQL databases... 3 found requesting database details for defaultdb... writing: debug/schema/defaultdb@details.json 0 tables found requesting database details for postgres... writing: debug/schema/postgres@details.json 0 tables found requesting database details for system... writing: debug/schema/system@details.json -29 tables found +30 tables found requesting table details for system.public.namespace... writing: debug/schema/system/public_namespace.json requesting table details for system.public.descriptor... writing: debug/schema/system/public_descriptor.json requesting table details for system.public.users... writing: debug/schema/system/public_users.json @@ -196,5 +198,6 @@ requesting table details for system.public.statement_diagnostics_requests... wri requesting table details for system.public.statement_diagnostics... writing: debug/schema/system/public_statement_diagnostics.json requesting table details for system.public.scheduled_jobs... writing: debug/schema/system/public_scheduled_jobs.json requesting table details for system.public.sqlliveness... writing: debug/schema/system/public_sqlliveness.json +requesting table details for system.public.migrations... writing: debug/schema/system/public_migrations.json writing: debug/pprof-summary.sh writing: debug/hot-ranges.sh diff --git a/pkg/cli/testdata/zip/partial2 b/pkg/cli/testdata/zip/partial2 index 7b0990d87880..caa801aab733 100644 --- a/pkg/cli/testdata/zip/partial2 +++ b/pkg/cli/testdata/zip/partial2 @@ -58,7 +58,7 @@ requesting goroutine files for node 1... writing: debug/nodes/1/goroutines.err.t ^- resulted in ... requesting log file ... requesting log file ... -requesting ranges... 35 found +requesting ranges... 36 found writing: debug/nodes/1/ranges/1.json writing: debug/nodes/1/ranges/2.json writing: debug/nodes/1/ranges/3.json @@ -94,6 +94,7 @@ writing: debug/nodes/1/ranges/32.json writing: debug/nodes/1/ranges/33.json writing: debug/nodes/1/ranges/34.json writing: debug/nodes/1/ranges/35.json +writing: debug/nodes/1/ranges/36.json writing: debug/nodes/3/status.json using SQL connection URL for node 3: postgresql://... retrieving SQL data for crdb_internal.feature_usage... writing: debug/nodes/3/crdb_internal.feature_usage.txt @@ -123,7 +124,7 @@ requesting goroutine files for node 3... writing: debug/nodes/3/goroutines.err.t ^- resulted in ... requesting log file ... requesting log file ... -requesting ranges... 35 found +requesting ranges... 36 found writing: debug/nodes/3/ranges/1.json writing: debug/nodes/3/ranges/2.json writing: debug/nodes/3/ranges/3.json @@ -159,13 +160,14 @@ writing: debug/nodes/3/ranges/32.json writing: debug/nodes/3/ranges/33.json writing: debug/nodes/3/ranges/34.json writing: debug/nodes/3/ranges/35.json +writing: debug/nodes/3/ranges/36.json requesting list of SQL databases... 3 found requesting database details for defaultdb... writing: debug/schema/defaultdb@details.json 0 tables found requesting database details for postgres... writing: debug/schema/postgres@details.json 0 tables found requesting database details for system... writing: debug/schema/system@details.json -29 tables found +30 tables found requesting table details for system.public.namespace... writing: debug/schema/system/public_namespace.json requesting table details for system.public.descriptor... writing: debug/schema/system/public_descriptor.json requesting table details for system.public.users... writing: debug/schema/system/public_users.json @@ -195,5 +197,6 @@ requesting table details for system.public.statement_diagnostics_requests... wri requesting table details for system.public.statement_diagnostics... writing: debug/schema/system/public_statement_diagnostics.json requesting table details for system.public.scheduled_jobs... writing: debug/schema/system/public_scheduled_jobs.json requesting table details for system.public.sqlliveness... writing: debug/schema/system/public_sqlliveness.json +requesting table details for system.public.migrations... writing: debug/schema/system/public_migrations.json writing: debug/pprof-summary.sh writing: debug/hot-ranges.sh diff --git a/pkg/cli/testdata/zip/specialnames b/pkg/cli/testdata/zip/specialnames index 62f9a077580b..66429360713e 100644 --- a/pkg/cli/testdata/zip/specialnames +++ b/pkg/cli/testdata/zip/specialnames @@ -21,7 +21,7 @@ requesting table details for defaultdb.public."../system"... writing: debug/sche requesting database details for postgres... writing: debug/schema/postgres@details.json 0 tables found requesting database details for system... writing: debug/schema/system-1@details.json -29 tables found +30 tables found requesting table details for system.public.namespace... writing: debug/schema/system-1/public_namespace.json requesting table details for system.public.descriptor... writing: debug/schema/system-1/public_descriptor.json requesting table details for system.public.users... writing: debug/schema/system-1/public_users.json @@ -51,3 +51,4 @@ requesting table details for system.public.statement_diagnostics_requests... wri requesting table details for system.public.statement_diagnostics... writing: debug/schema/system-1/public_statement_diagnostics.json requesting table details for system.public.scheduled_jobs... writing: debug/schema/system-1/public_scheduled_jobs.json requesting table details for system.public.sqlliveness... writing: debug/schema/system-1/public_sqlliveness.json +requesting table details for system.public.migrations... writing: debug/schema/system-1/public_migrations.json diff --git a/pkg/cli/testdata/zip/testzip b/pkg/cli/testdata/zip/testzip index 8cb60a5e3f69..713a98205c13 100644 --- a/pkg/cli/testdata/zip/testzip +++ b/pkg/cli/testdata/zip/testzip @@ -57,7 +57,7 @@ requesting heap profile for node 1... writing: debug/nodes/1/heap.pprof requesting heap files for node 1... ? found requesting goroutine files for node 1... 0 found requesting log file ... -requesting ranges... 35 found +requesting ranges... 36 found writing: debug/nodes/1/ranges/1.json writing: debug/nodes/1/ranges/2.json writing: debug/nodes/1/ranges/3.json @@ -93,13 +93,14 @@ writing: debug/nodes/1/ranges/32.json writing: debug/nodes/1/ranges/33.json writing: debug/nodes/1/ranges/34.json writing: debug/nodes/1/ranges/35.json +writing: debug/nodes/1/ranges/36.json requesting list of SQL databases... 3 found requesting database details for defaultdb... writing: debug/schema/defaultdb@details.json 0 tables found requesting database details for postgres... writing: debug/schema/postgres@details.json 0 tables found requesting database details for system... writing: debug/schema/system@details.json -29 tables found +30 tables found requesting table details for system.public.namespace... writing: debug/schema/system/public_namespace.json requesting table details for system.public.descriptor... writing: debug/schema/system/public_descriptor.json requesting table details for system.public.users... writing: debug/schema/system/public_users.json @@ -129,5 +130,6 @@ requesting table details for system.public.statement_diagnostics_requests... wri requesting table details for system.public.statement_diagnostics... writing: debug/schema/system/public_statement_diagnostics.json requesting table details for system.public.scheduled_jobs... writing: debug/schema/system/public_scheduled_jobs.json requesting table details for system.public.sqlliveness... writing: debug/schema/system/public_sqlliveness.json +requesting table details for system.public.migrations... writing: debug/schema/system/public_migrations.json writing: debug/pprof-summary.sh writing: debug/hot-ranges.sh diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index 2a8f1abcf3b4..8bf2433caf89 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -199,6 +199,16 @@ const ( // EmptyArraysInInvertedIndexes is when empty arrays are added to array // inverted indexes. EmptyArraysInInvertedIndexes + // MigrationTable introduces the new system.migrations table. + MigrationTable + // TruncatedAndRangeAppliedStateMigration is part of the migration to stop + // using the legacy truncated state within KV. Once it's active, we'll be + // using the unreplicated truncated state and the RangeAppliedState on all + // ranges. In 21.2 we'll now be able to remove any holdover code handling + // the possibility of replicated truncated state. + // + // TODO(irfansharif): Do the above in 21.2. + TruncatedAndRangeAppliedStateMigration // Step (1): Add new versions here. ) @@ -321,6 +331,14 @@ var versionsSingleton = keyedVersions([]keyedVersion{ Key: EmptyArraysInInvertedIndexes, Version: roachpb.Version{Major: 20, Minor: 2, Internal: 4}, }, + { + Key: MigrationTable, + Version: roachpb.Version{Major: 20, Minor: 2, Internal: 6}, + }, + { + Key: TruncatedAndRangeAppliedStateMigration, + Version: roachpb.Version{Major: 20, Minor: 2, Internal: 8}, + }, // Step (2): Add new versions here. }) diff --git a/pkg/clusterversion/key_string.go b/pkg/clusterversion/key_string.go index 277d6bb36297..22b6e857516b 100644 --- a/pkg/clusterversion/key_string.go +++ b/pkg/clusterversion/key_string.go @@ -32,11 +32,13 @@ func _() { _ = x[V20_2-21] _ = x[Start21_1-22] _ = x[EmptyArraysInInvertedIndexes-23] + _ = x[MigrationTable-24] + _ = x[TruncatedAndRangeAppliedStateMigration-25] } -const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesClientRangeInfosOnBatchResponseNodeMembershipStatusRangeStatsRespHasDescMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexes" +const _Key_name = "NamespaceTableWithSchemasStart20_2GeospatialTypeEnumsRangefeedLeasesAlterColumnTypeGeneralAlterSystemJobsAddCreatedByColumnsAddScheduledJobsTableUserDefinedSchemasNoOriginFKIndexesClientRangeInfosOnBatchResponseNodeMembershipStatusRangeStatsRespHasDescMinPasswordLengthAbortSpanBytesAlterSystemJobsAddSqllivenessColumnsAddNewSystemSqllivenessTableMaterializedViewsBox2DTypeUpdateScheduledJobsSchemaCreateLoginPrivilegeHBAForNonTLSV20_2Start21_1EmptyArraysInInvertedIndexesMigrationTableTruncatedAndRangeAppliedStateMigration" -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} +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, 486, 524} 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 96c23bc917b9..374a776fd42e 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -390,6 +390,7 @@ const ( ScheduledJobsTableID = 37 TenantsRangesID = 38 // pseudo SqllivenessID = 39 + MigrationsID = 40 // CommentType is type for system.comments DatabaseCommentType = 0 diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index 098d9336d619..56146074fabe 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -227,19 +227,11 @@ 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 + // Nothing to do for the methods below as they do not generate // any rows. case *roachpb.EndTxnRequest: case *roachpb.AdminMergeRequest: @@ -254,6 +246,7 @@ func (b *Batch) fillResults(ctx context.Context) { case *roachpb.PushTxnRequest: case *roachpb.QueryTxnRequest: case *roachpb.QueryIntentRequest: + case *roachpb.MigrateRequest: case *roachpb.ResolveIntentRequest: case *roachpb.ResolveIntentRangeRequest: case *roachpb.MergeRequest: @@ -264,6 +257,11 @@ func (b *Batch) fillResults(ctx context.Context) { case *roachpb.ImportRequest: case *roachpb.AdminScatterRequest: case *roachpb.AddSSTableRequest: + default: + if result.Err == nil { + result.Err = errors.Errorf("unsupported reply: %T for %T", + reply, args) + } } // Fill up the resume span. if result.Err == nil && reply != nil && reply.Header().ResumeSpan != nil { @@ -754,3 +752,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 89a2c001e695..9d9b9b95c8ee 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -639,6 +639,15 @@ func (db *DB) AddSSTable( return getOneErr(db.Run(ctx, b), b) } +// Migrate proactively forces ranges overlapping with the provided keyspace to +// transition out of any legacy modes of operation (as defined by the target +// version). +func (db *DB) Migrate(ctx context.Context, begin, end interface{}, version roachpb.Version) error { + b := &Batch{} + b.migrate(begin, end, version) + return getOneErr(db.Run(ctx, b), b) +} + // sendAndFill is a helper which sends the given batch and fills its results, // returning the appropriate error which is either from the first failing call, // or an "internal" error. diff --git a/pkg/kv/kvserver/batcheval/BUILD.bazel b/pkg/kv/kvserver/batcheval/BUILD.bazel index 77a5c3c86544..20f81184284b 100644 --- a/pkg/kv/kvserver/batcheval/BUILD.bazel +++ b/pkg/kv/kvserver/batcheval/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "cmd_lease_request.go", "cmd_lease_transfer.go", "cmd_merge.go", + "cmd_migrate.go", "cmd_push_txn.go", "cmd_put.go", "cmd_query_intent.go", diff --git a/pkg/kv/kvserver/batcheval/cmd_migrate.go b/pkg/kv/kvserver/batcheval/cmd_migrate.go new file mode 100644 index 000000000000..752c75d9b950 --- /dev/null +++ b/pkg/kv/kvserver/batcheval/cmd_migrate.go @@ -0,0 +1,107 @@ +// 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( + _ *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.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. +func Migrate( + ctx context.Context, readWriter storage.ReadWriter, cArgs CommandArgs, _ roachpb.Response, +) (result.Result, error) { + args := cArgs.Args.(*roachpb.MigrateRequest) + + fn, ok := migrationRegistry[args.Version] + if !ok { + return result.Result{}, errors.Newf("migration for %s not found", args.Version) + } + return fn(ctx, readWriter, cArgs) +} + +// 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 +} diff --git a/pkg/kv/kvserver/queue_helpers_testutil.go b/pkg/kv/kvserver/queue_helpers_testutil.go index 31cf64f7154f..25d95be83497 100644 --- a/pkg/kv/kvserver/queue_helpers_testutil.go +++ b/pkg/kv/kvserver/queue_helpers_testutil.go @@ -14,7 +14,6 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/errors" ) // Code in this file is for testing usage only. It is exported only because it @@ -26,23 +25,6 @@ func (bq *baseQueue) testingAdd( return bq.addInternal(ctx, repl.Desc(), repl.ReplicaID(), priority) } -func forceScanAndProcess(s *Store, q *baseQueue) error { - // Check that the system config is available. It is needed by many queues. If - // it's not available, some queues silently fail to process any replicas, - // which is undesirable for this method. - if cfg := s.Gossip().GetSystemConfig(); cfg == nil { - return errors.Errorf("system config not available in gossip") - } - - newStoreReplicaVisitor(s).Visit(func(repl *Replica) bool { - q.maybeAdd(context.Background(), repl, s.cfg.Clock.Now()) - return true - }) - - q.DrainQueue(s.stopper) - return nil -} - func mustForceScanAndProcess(ctx context.Context, s *Store, q *baseQueue) { if err := forceScanAndProcess(s, q); err != nil { log.Fatalf(ctx, "%v", err) @@ -55,6 +37,12 @@ func (s *Store) ForceReplicationScanAndProcess() error { return forceScanAndProcess(s, s.replicateQueue.baseQueue) } +// ForceReplicaGCScanAndProcess iterates over all ranges and enqueues any that +// may need to be GC'd. +func (s *Store) ForceReplicaGCScanAndProcess() error { + return forceScanAndProcess(s, s.replicaGCQueue.baseQueue) +} + // MustForceReplicaGCScanAndProcess iterates over all ranges and enqueues any that // may need to be GC'd. func (s *Store) MustForceReplicaGCScanAndProcess() { diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index 0fbd20a1693d..965866bdf833 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -773,12 +773,14 @@ func (r *Replica) evaluateProposal( usingAppliedStateKey := r.mu.state.UsingAppliedStateKey r.mu.RUnlock() if !usingAppliedStateKey { - // 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. + // The range applied state was introduced in v21.1. The cluster + // version transition into v21.1 ought to have migrated any holdover + // ranges still using the legacy keys, which is what we assert + // below. If we're not running 21.1 yet, migrate over as we've done + // since the introduction of the applied state key. + if r.ClusterSettings().Version.IsActive(ctx, clusterversion.TruncatedAndRangeAppliedStateMigration) { + log.Fatalf(ctx, "not using applied state key in v20.1") + } if res.Replicated.State == nil { res.Replicated.State = &kvserverpb.ReplicaState{} } diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index 1c8edfceec9e..7d3116945787 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -216,6 +216,27 @@ func (r *Replica) executeWriteBatch( log.Warningf(ctx, "%v", err) } } + if ba.Requests[0].GetMigrate() != nil && propResult.Err == nil { + // Migrate is special since it wants commands to be durably + // applied on all peers, which we achieve via waitForApplication. + // + // NB: We don't have to worry about extant snapshots creating + // replicas that start at an index before this Migrate request. + // Snapshots that don't include the recipient (as specified by + // replicaID and descriptor in the snap vs. the replicaID of the + // raft instance) are discarded by the recipient, and we're + // already checking against all replicas in the descriptor + // below. Snapshots are also discarded unless they move the LAI + // forward, so we're not worried about old snapshots (with + // indexes preceding the MLAI here) instantiating pre-migrated + // state in anyway. + desc := r.Desc() + // NB: waitForApplication already has a timeout. + applicationErr := waitForApplication( + ctx, r.store.cfg.NodeDialer, desc.RangeID, desc.Replicas().All(), + uint64(maxLeaseIndex)) + propResult.Err = roachpb.NewError(applicationErr) + } return propResult.Reply, nil, propResult.Err case <-slowTimer.C: slowTimer.Read = true diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 2dc51826f75e..64a7eafbf291 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -2765,6 +2765,32 @@ func (s *Store) ManuallyEnqueue( return collect(), processErr, nil } +// GCReplicas iterates over all ranges and processes any that may need to be +// GC'd. +func (s *Store) GCReplicas() error { + if interceptor := s.TestingKnobs().GCReplicasInterceptor; interceptor != nil { + interceptor() + } + return forceScanAndProcess(s, s.replicaGCQueue.baseQueue) +} + +func forceScanAndProcess(s *Store, q *baseQueue) error { + // Check that the system config is available. It is needed by many queues. If + // it's not available, some queues silently fail to process any replicas, + // which is undesirable for this method. + if cfg := s.Gossip().GetSystemConfig(); cfg == nil { + return errors.Errorf("system config not available in gossip") + } + + newStoreReplicaVisitor(s).Visit(func(repl *Replica) bool { + q.maybeAdd(context.Background(), repl, s.cfg.Clock.Now()) + return true + }) + + q.DrainQueue(s.stopper) + return nil +} + // WriteClusterVersion writes the given cluster version to the store-local // cluster version key. func WriteClusterVersion( diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index dcad8b9988da..bfc268cabb3e 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -255,6 +255,9 @@ type StoreTestingKnobs struct { // heartbeats and then expect other replicas to take the lease without // worrying about Raft). AllowLeaseRequestProposalsWhenNotLeader bool + // GCReplicasInterceptor intercepts attempts to GC all replicas in the + // store. + GCReplicasInterceptor func() } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. diff --git a/pkg/migration/BUILD.bazel b/pkg/migration/BUILD.bazel index c02fd0142384..b639857f621c 100644 --- a/pkg/migration/BUILD.bazel +++ b/pkg/migration/BUILD.bazel @@ -1,8 +1,9 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "migration", srcs = [ + "helper.go", "manager.go", "migrations.go", "util.go", @@ -11,6 +12,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/clusterversion", + "//pkg/keys", "//pkg/kv", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", @@ -18,8 +20,45 @@ go_library( "//pkg/rpc/nodedialer", "//pkg/server/serverpb", "//pkg/sql", + "//pkg/sql/sqlutil", + "//pkg/util/ctxgroup", "//pkg/util/log", + "//pkg/util/timeutil", "//vendor/github.com/cockroachdb/errors", "//vendor/github.com/cockroachdb/logtags", + "//vendor/github.com/cockroachdb/redact", + "//vendor/google.golang.org/grpc", + ], +) + +go_test( + name = "migration_test", + srcs = [ + "client_test.go", + "helper_test.go", + "main_test.go", + "manager_test.go", + "util_test.go", + ], + embed = [":migration"], + deps = [ + "//pkg/clusterversion", + "//pkg/kv", + "//pkg/kv/kvserver", + "//pkg/kv/kvserver/liveness/livenesspb", + "//pkg/roachpb", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/server/serverpb", + "//pkg/sql/sqlutil", + "//pkg/sql/tests", + "//pkg/testutils", + "//pkg/testutils/serverutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "//pkg/util/syncutil", + "//vendor/github.com/cockroachdb/errors", + "//vendor/google.golang.org/grpc", ], ) diff --git a/pkg/migration/client_test.go b/pkg/migration/client_test.go new file mode 100644 index 000000000000..82b253001f22 --- /dev/null +++ b/pkg/migration/client_test.go @@ -0,0 +1,171 @@ +package migration_test + +import ( + "context" + "database/sql" + "testing" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/migration" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/sql/tests" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" +) + +func TestHelperIterateRangeDescriptors(t *testing.T) { + defer leaktest.AfterTest(t) + + cv := clusterversion.ClusterVersion{} + ctx := context.Background() + const numNodes = 1 + + params, _ := tests.CreateTestServerParams() + server, _, kvDB := serverutils.StartServer(t, params) + defer server.Stopper().Stop(context.Background()) + + var numRanges int + if err := server.GetStores().(*kvserver.Stores).VisitStores(func(s *kvserver.Store) error { + numRanges = s.ReplicaCount() + return nil + }); err != nil { + t.Fatal(err) + } + + c := migration.TestingNewCluster(numNodes, kvDB, server.InternalExecutor().(sqlutil.InternalExecutor)) + h := migration.TestingNewHelper(c, cv) + + for _, blockSize := range []int{1, 5, 10, 50} { + var numDescs int + init := func() { numDescs = 0 } + if err := h.IterateRangeDescriptors(ctx, blockSize, init, func(descriptors ...roachpb.RangeDescriptor) error { + numDescs += len(descriptors) + return nil + }); err != nil { + t.Fatal(err) + } + + // TODO(irfansharif): We always seem to include a second copy of r1's + // desc. Unsure why. + if numDescs != numRanges+1 { + t.Fatalf("expected to find %d ranges, found %d", numRanges+1, numDescs) + } + + } +} + +func TestHelperWithMigrationTable(t *testing.T) { + defer leaktest.AfterTest(t) + + // Sort above any real version. + cv := clusterversion.ClusterVersion{ + Version: roachpb.Version{Major: 420, Minor: 7, Internal: 10}, + } + ctx := context.Background() + const numNodes = 1 + + params, _ := tests.CreateTestServerParams() + server, sqlDB, kvDB := serverutils.StartServer(t, params) + defer server.Stopper().Stop(context.Background()) + + c := migration.TestingNewCluster(numNodes, kvDB, server.InternalExecutor().(sqlutil.InternalExecutor)) + h := migration.TestingNewHelper(c, cv) + + dummyDesc := "dummy desc" + if err := h.TestingInsertMigrationRecord(ctx, "dummy desc"); err != nil { + t.Fatal(err) + } + { + // Check to see that the initial migration record is what we expect. + row := sqlDB.QueryRow(` + SELECT version, status, description + FROM system.migrations + ORDER BY version DESC LIMIT 1`, + ) + var version, status, desc string + if err := row.Scan(&version, &status, &desc); err != nil { + t.Fatal(err) + } + + if version != cv.String() { + t.Fatalf("expected %s, got %s", cv, version) + } + + if status != string(migration.StatusRunning) { + t.Fatalf("expected %s, got %s", migration.StatusSucceeded, status) + } + + if desc != dummyDesc { + t.Fatalf("expected %s, got %s", dummyDesc, desc) + } + } + + dummyProgress := "dummy progress" + if err := h.UpdateProgress(ctx, dummyProgress); err != nil { + t.Fatal(err) + } + + { + row := sqlDB.QueryRow(` + SELECT progress + FROM system.migrations + ORDER BY version DESC LIMIT 1`, + ) + var progress string + if err := row.Scan(&progress); err != nil { + t.Fatal(err) + } + + if progress != dummyProgress { + t.Fatalf("expected %s, got %s", dummyProgress, progress) + } + } + + if err := h.TestingUpdateStatus(ctx, migration.StatusFailed); err != nil { + t.Fatal(err) + } + + { + row := sqlDB.QueryRow(` + SELECT status, completed + FROM system.migrations + ORDER BY version DESC LIMIT 1`, + ) + var status string + var completed sql.NullTime + if err := row.Scan(&status, &completed); err != nil { + t.Fatal(err) + } + + if status != string(migration.StatusFailed) { + t.Fatalf("expected %s, got %s", dummyProgress, migration.StatusFailed) + } + if (completed != sql.NullTime{}) { + t.Fatalf("expected empty completed timestamp, got %v", completed) + } + } + if err := h.TestingUpdateStatus(ctx, migration.StatusSucceeded); err != nil { + t.Fatal(err) + } + { + row := sqlDB.QueryRow(` + SELECT status, completed + FROM system.migrations + ORDER BY version DESC LIMIT 1`, + ) + var status string + var completed sql.NullTime + if err := row.Scan(&status, &completed); err != nil { + t.Fatal(err) + } + + if status != string(migration.StatusSucceeded) { + t.Fatalf("expected %s, got %s", dummyProgress, migration.StatusSucceeded) + } + if (completed == sql.NullTime{}) { + t.Fatalf("expected non-empty completed timestamp") + } + } +} diff --git a/pkg/migration/helper.go b/pkg/migration/helper.go new file mode 100644 index 000000000000..f05fa440c10f --- /dev/null +++ b/pkg/migration/helper.go @@ -0,0 +1,349 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package migration + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" + "google.golang.org/grpc" +) + +// Helper captures all the primitives required to fully specify a migration. +type Helper struct { + c cluster + cv clusterversion.ClusterVersion +} + +// cluster mediates access to the crdb cluster. +type cluster interface { + // nodes returns the IDs and epochs for all nodes that are currently part of + // the cluster (i.e. they haven't been decommissioned away). Migrations have + // the pre-requisite that all nodes are up and running so that we're able to + // execute all relevant node-level operations on them. If any of the nodes + // are found to be unavailable, an error is returned. + // + // It's important to note that this makes no guarantees about new nodes + // being added to the cluster. It's entirely possible for that to happen + // concurrently with the retrieval of the current set of nodes. Appropriate + // usage of this entails wrapping it under a stabilizing loop, like we do in + // EveryNode. + nodes(ctx context.Context) (nodes, error) + + // dial returns a grpc connection to the given node. + dial(context.Context, roachpb.NodeID) (*grpc.ClientConn, error) + + // db provides access the kv.DB instance backing the cluster. + // + // TODO(irfansharif): We could hide the kv.DB instance behind an interface + // to expose only relevant, vetted bits of kv.DB. It'll make our tests less + // "integration-ey". + db() *kv.DB + + // executor provides access to an internal executor instance to run + // arbitrary SQL statements. + executor() sqlutil.InternalExecutor +} + +func newHelper(c cluster, cv clusterversion.ClusterVersion) *Helper { + return &Helper{c: c, cv: cv} +} + +// IterateRangeDescriptors provides a handle on every range descriptor in the +// system, which callers can then use to send out arbitrary KV requests to in +// order to run arbitrary KV-level migrations. These requests will typically +// just be the `Migrate` request, with code added within [1] to do the specific +// things intended for the specified version. +// +// It's important to note that the closure is being executed in the context of a +// distributed transaction that may be automatically retried. So something like +// the following is an anti-pattern: +// +// processed := 0 +// _ = h.IterateRangeDescriptors(..., +// func(descriptors ...roachpb.RangeDescriptor) error { +// processed += len(descriptors) // we'll over count if retried +// log.Infof(ctx, "processed %d ranges", processed) +// }, +// ) +// +// Instead we allow callers to pass in a callback to signal on every attempt +// (including the first). This lets us salvage the example above: +// +// var processed int +// init := func() { processed = 0 } +// _ = h.IterateRangeDescriptors(..., init, +// func(descriptors ...roachpb.RangeDescriptor) error { +// processed += len(descriptors) +// log.Infof(ctx, "processed %d ranges", processed) +// }, +// ) +// +// [1]: pkg/kv/kvserver/batch_eval/cmd_migrate.go +func (h *Helper) IterateRangeDescriptors( + ctx context.Context, blockSize int, init func(), fn func(...roachpb.RangeDescriptor) error, +) error { + if err := h.c.db().Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + // Inform the caller that we're starting a fresh attempt to page in + // range descriptors. + init() + + // Iterate through the meta ranges to pull out all the range descriptors. + return txn.Iterate(ctx, keys.MetaMin, keys.MetaMax, blockSize, + func(rows []kv.KeyValue) error { + descriptors := make([]roachpb.RangeDescriptor, len(rows)) + for i, row := range rows { + if err := row.ValueProto(&descriptors[i]); err != nil { + return errors.Wrapf(err, + "unable to unmarshal range descriptor from %s", + row.Key, + ) + } + } + + // Invoke fn with the current chunk (of size ~blockSize) of + // range descriptors. + if err := fn(descriptors...); err != nil { + return err + } + + return nil + }) + }); err != nil { + return err + } + + return nil +} + +// 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( + ctx context.Context, op string, fn func(context.Context, serverpb.MigrationClient) error, +) error { + ns, err := h.c.nodes(ctx) + if err != nil { + return err + } + + for { + log.Infof(ctx, "executing %s on nodes %s", op, ns) + + grp := ctxgroup.WithContext(ctx) + for _, node := range ns { + grp.GoCtx(func(ctx context.Context) error { + conn, err := h.c.dial(ctx, node.id) + if err != nil { + return err + } + client := serverpb.NewMigrationClient(conn) + if err := fn(ctx, client); err != nil { + return err + } + return nil + }) + } + if err := grp.Wait(); err != nil { + return err + } + + curNodes, err := h.c.nodes(ctx) + if err != nil { + return err + } + + if ok, diff := ns.identical(curNodes); !ok { + log.Infof(ctx, "%s, retrying", diff) + ns = curNodes + continue + } + + break + } + + return nil +} + +// UpdateProgress is used to update the externally visible progress indicator +// (accessible using the `progress` column in `system.migrations`) for the +// ongoing migration. +func (h *Helper) UpdateProgress(ctx context.Context, progress string) error { + const recordProgressStmt = ` +UPDATE system.migrations SET "progress" = $1 WHERE "version" = $2 +` + args := []interface{}{progress, h.cv.String()} + _, err := h.c.executor().Exec(ctx, "update-progress", + nil, recordProgressStmt, args...) + if err != nil { + return err + } + return nil +} + +// ClusterVersion exposes the cluster version associated with the ongoing +// migration. +func (h *Helper) ClusterVersion() clusterversion.ClusterVersion { + return h.cv +} + +// DB provides exposes the underlying *kv.DB instance. +func (h *Helper) DB() *kv.DB { + return h.c.db() +} + +// insertMigrationRecord creates a record in the `system.migrations` table for +// the specific migration. This record provides external observability into the +// state of the migration. It indicates whether the specific migration is +// ongoing, whether it has succeeded or failed, what the most recent progress +// update was, and error from the last failed attempt (if any). +// +// NB: This only intended user of this is the migration infrastructure itself, +// not authored migrations. +func (h *Helper) insertMigrationRecord(ctx context.Context, desc string) error { + // XXX: The ON CONFLICT clause is a stop gap for until I add real tests. + // Right now I'm allowing the infrastructure to re-do migrations over and + // over. + const insertMigrationRecordStmt = ` +INSERT into system.migrations ("version", "status", "description") +VALUES ($1, $2, $3) ON CONFLICT ("version") DO UPDATE SET "start" = now() +` + args := []interface{}{ + h.cv.String(), + StatusRunning, + desc, + } + + _, err := h.c.executor().Exec(ctx, "insert-migration", + nil, insertMigrationRecordStmt, args...) + if err != nil { + return err + } + return nil +} + +// updateStatus is used to update the externally visible status indicator +// (accessible using the `status` column in `system.migrations`) for the +// ongoing migration. +// +// NB: This only intended user of this is the migration infrastructure itself, +// not authored migrations. +func (h *Helper) updateStatus(ctx context.Context, status Status) error { + const updateStatusStmt = ` +UPDATE system.migrations SET ("status", "completed") = ($1, $2) +WHERE "version" = $3 +` + args := []interface{}{ + status, + nil, // completed + h.ClusterVersion().String(), + } + + if status == StatusSucceeded { + args[1] = timeutil.Now() + } + _, err := h.c.executor().Exec(ctx, "update-status", + nil, updateStatusStmt, args...) + if err != nil { + return err + } + return nil +} + +type clusterImpl struct { + nl nodeLiveness + exec sqlutil.InternalExecutor + dialer *nodedialer.Dialer + kvDB *kv.DB +} + +var _ cluster = &clusterImpl{} + +// nodes implements the cluster interface. +func (c *clusterImpl) nodes(ctx context.Context) (nodes, error) { + var ns []node + ls, err := c.nl.GetLivenessesFromKV(ctx) + if err != nil { + return nil, err + } + for _, l := range ls { + if l.Membership.Decommissioned() { + continue + } + live, err := c.nl.IsLive(l.NodeID) + if err != nil { + return nil, err + } + if !live { + return nil, errors.Newf("n%d required, but unavailable", l.NodeID) + } + ns = append(ns, node{id: l.NodeID, epoch: l.Epoch}) + } + return ns, nil +} + +// dial implements the cluster interface. +func (c *clusterImpl) dial(ctx context.Context, id roachpb.NodeID) (*grpc.ClientConn, error) { + return c.dialer.Dial(ctx, id, rpc.DefaultClass) +} + +// db implements the cluster interface. +func (c *clusterImpl) db() *kv.DB { + return c.kvDB +} + +// executor implements the cluster interface. +func (c *clusterImpl) executor() sqlutil.InternalExecutor { + return c.exec +} diff --git a/pkg/migration/helper_test.go b/pkg/migration/helper_test.go new file mode 100644 index 000000000000..99498b8485d8 --- /dev/null +++ b/pkg/migration/helper_test.go @@ -0,0 +1,300 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package migration + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/errors" + "google.golang.org/grpc" +) + +func TestHelperEveryNode(t *testing.T) { + defer leaktest.AfterTest(t) + + cv := clusterversion.ClusterVersion{} + ctx := context.Background() + var mu syncutil.Mutex + const numNodes = 3 + + t.Run("with-node-addition", func(t *testing.T) { + // Add a node mid-way through execution. We expect EveryNode to start + // over from scratch and include the newly added node. + tc := TestingNewCluster(numNodes, nil, nil) + h := newHelper(tc, cv) + opCount := 0 + err := h.EveryNode(ctx, "dummy-op", func(context.Context, serverpb.MigrationClient) error { + mu.Lock() + defer mu.Unlock() + + opCount += 1 + if opCount == numNodes { + tc.addNode() + } + + return nil + }) + if err != nil { + t.Fatal(err) + } + + if exp := numNodes*2 + 1; exp != opCount { + t.Fatalf("expected closure to be invoked %d times, got %d", exp, opCount) + } + }) + + t.Run("with-node-restart", func(t *testing.T) { + // Restart a node mid-way through execution. We expect EveryNode to + // start over from scratch and include the restarted node. + tc := TestingNewCluster(numNodes, nil, nil) + h := newHelper(tc, cv) + opCount := 0 + err := h.EveryNode(ctx, "dummy-op", func(context.Context, serverpb.MigrationClient) error { + mu.Lock() + defer mu.Unlock() + + opCount += 1 + if opCount == numNodes { + tc.restartNode(2) + } + + return nil + }) + if err != nil { + t.Fatal(err) + } + + if exp := numNodes * 2; exp != opCount { + t.Fatalf("expected closure to be invoked %d times, got %d", exp, opCount) + } + }) + + t.Run("with-node-down", func(t *testing.T) { + // Down a node mid-way through execution. We expect EveryNode to error + // out. + const downedNode = 2 + tc := TestingNewCluster(numNodes, nil, nil) + tc.downNode(downedNode) + expRe := fmt.Sprintf("n%d required, but unavailable", downedNode) + h := newHelper(tc, cv) + if err := h.EveryNode(ctx, "dummy-op", func(context.Context, serverpb.MigrationClient) error { + return nil + }); !testutils.IsError(err, expRe) { + t.Fatalf("expected error %q, got %q", expRe, err) + } + + tc.healCluster() + if err := h.EveryNode(ctx, "dummy-op", func(context.Context, serverpb.MigrationClient) error { + return nil + }); err != nil { + t.Fatal(err) + } + }) +} + +func TestClusterNodes(t *testing.T) { + defer leaktest.AfterTest(t) + + ctx := context.Background() + const numNodes = 3 + + t.Run("retrieve-all", func(t *testing.T) { + nl := newTestNodeLiveness(numNodes) + c := clusterImpl{nl: nl} + + ns, err := c.nodes(ctx) + if err != nil { + t.Fatal(err) + } + + if got := len(ns); got != numNodes { + t.Fatalf("expected %d nodes, got %d", numNodes, got) + } + + for i := range ns { + if exp := roachpb.NodeID(i + 1); exp != ns[i].id { + t.Fatalf("expected to find node ID %s, got %s", exp, ns[i].id) + } + if ns[i].epoch != 1 { + t.Fatalf("expected to find epoch=1, got %d", ns[i].epoch) + } + } + }) + + t.Run("ignore-decommissioned", func(t *testing.T) { + nl := newTestNodeLiveness(numNodes) + c := clusterImpl{nl: nl} + const decommissionedNode = 3 + nl.decommission(decommissionedNode) + + ns, err := c.nodes(ctx) + if err != nil { + t.Fatal(err) + } + + if got := len(ns); got != numNodes-1 { + t.Fatalf("expected %d nodes, got %d", numNodes-1, got) + } + + for i := range ns { + if exp := roachpb.NodeID(i + 1); exp != ns[i].id { + t.Fatalf("expected to find node ID %s, got %s", exp, ns[i].id) + } + if ns[i].epoch != 1 { + t.Fatalf("expected to find epoch=1, got %d", ns[i].epoch) + } + } + }) + + t.Run("error-if-down", func(t *testing.T) { + nl := newTestNodeLiveness(numNodes) + c := clusterImpl{nl: nl} + const downedNode = 3 + nl.down(downedNode) + + _, err := c.nodes(ctx) + expRe := fmt.Sprintf("n%d required, but unavailable", downedNode) + if !testutils.IsError(err, expRe) { + t.Fatalf("expected error %q, got %q", expRe, err) + } + }) +} + +type testClusterImpl struct { + ns nodes + err error + kvDB *kv.DB + exec sqlutil.InternalExecutor +} + +var _ cluster = &testClusterImpl{} + +func TestingNewCluster(numNodes int, db *kv.DB, exec sqlutil.InternalExecutor) *testClusterImpl { + tc := &testClusterImpl{ + kvDB: db, + exec: exec, + } + for i := 0; i < numNodes; i++ { + tc.addNode() + } + + return tc +} + +func (t *testClusterImpl) nodes(context.Context) (nodes, error) { + ns := make([]node, len(t.ns)) + for i, n := range t.ns { + ns[i] = n + } + return ns, t.err +} + +func (t *testClusterImpl) dial(context.Context, roachpb.NodeID) (*grpc.ClientConn, error) { + return nil, nil +} + +func (t *testClusterImpl) db() *kv.DB { + return t.kvDB +} + +func (t *testClusterImpl) executor() sqlutil.InternalExecutor { + return t.exec +} + +func (t *testClusterImpl) addNode() { + t.ns = append(t.ns, node{id: roachpb.NodeID(len(t.ns) + 1), epoch: 1}) +} + +func (t *testClusterImpl) restartNode(id int) { + for i := range t.ns { + if t.ns[i].id == roachpb.NodeID(id) { + t.ns[i].epoch++ + break + } + } +} + +func (t *testClusterImpl) downNode(id int) { + t.err = errors.Newf("n%d required, but unavailable", id) +} + +func (t *testClusterImpl) healCluster() { + t.err = nil +} + +type testNodeLivenessImpl struct { + ls []livenesspb.Liveness + dead map[roachpb.NodeID]struct{} +} + +var _ nodeLiveness = &testNodeLivenessImpl{} + +func newTestNodeLiveness(numNodes int) *testNodeLivenessImpl { + nl := &testNodeLivenessImpl{ + ls: make([]livenesspb.Liveness, numNodes), + dead: make(map[roachpb.NodeID]struct{}), + } + for i := 0; i < numNodes; i++ { + nl.ls[i] = livenesspb.Liveness{ + NodeID: roachpb.NodeID(i + 1), + Epoch: 1, + Membership: livenesspb.MembershipStatus_ACTIVE, + } + } + return nl +} + +func (t *testNodeLivenessImpl) GetLivenessesFromKV( + ctx context.Context, +) ([]livenesspb.Liveness, error) { + return t.ls, nil +} + +func (t *testNodeLivenessImpl) IsLive(id roachpb.NodeID) (bool, error) { + _, dead := t.dead[id] + return !dead, nil +} + +func (t *testNodeLivenessImpl) decommission(id roachpb.NodeID) { + for i := range t.ls { + if t.ls[i].NodeID == id { + t.ls[i].Membership = livenesspb.MembershipStatus_DECOMMISSIONED + break + } + } +} + +func (t *testNodeLivenessImpl) down(id roachpb.NodeID) { + t.dead[id] = struct{}{} +} + +func TestingNewHelper(c cluster, cv clusterversion.ClusterVersion) *Helper { + return &Helper{c: c, cv: cv} +} + +func (h *Helper) TestingInsertMigrationRecord(ctx context.Context, desc string) error { + return h.insertMigrationRecord(ctx, desc) +} + +func (h *Helper) TestingUpdateStatus(ctx context.Context, status Status) error { + return h.updateStatus(ctx, status) +} diff --git a/pkg/migration/main_test.go b/pkg/migration/main_test.go new file mode 100644 index 000000000000..cdc3f7742a5a --- /dev/null +++ b/pkg/migration/main_test.go @@ -0,0 +1,29 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package migration_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" +) + +func TestMain(m *testing.M) { + security.SetAssetLoader(securitytest.EmbeddedAssets) + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} diff --git a/pkg/migration/manager.go b/pkg/migration/manager.go index 25488e7ec2a1..f0016746fa61 100644 --- a/pkg/migration/manager.go +++ b/pkg/migration/manager.go @@ -29,147 +29,23 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/server/serverpb" - "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" ) -// Migration defines a program to be executed once every node in the cluster is -// (a) running a specific binary version, and (b) has completed all prior -// migrations. -// -// Each migration is associated with a specific internal cluster version and is -// idempotent in nature. When setting the cluster version (via `SET CLUSTER -// SETTING version`), a manager process determines the set of migrations needed -// to bridge the gap between the current active cluster version, and the target -// one. -// -// To introduce a migration, introduce a version key in pkg/clusterversion, and -// introduce a corresponding internal cluster version for it. See [1] for -// details. Following that, define a Migration in this package and add it to the -// Registry. Be sure to key it in with the new cluster version we just added. -// During cluster upgrades, once the operator is able to set a cluster version -// setting that's past the version that was introduced (typically the major -// release version the migration was introduced in), the manager will execute -// the defined migration before letting the upgrade finalize. -// -// [1]: pkg/clusterversion/cockroach_versions.go -type Migration func(context.Context, *Helper) error - // Manager is the instance responsible for executing migrations across the // cluster. type Manager struct { dialer *nodedialer.Dialer + executor sqlutil.InternalExecutor nl nodeLiveness - executor *sql.InternalExecutor db *kv.DB } -// Helper captures all the primitives required to fully specify a migration. -type Helper struct { - *Manager -} - -// RequiredNodeIDs returns the node IDs for all nodes that are currently part of -// the cluster (i.e. they haven't been decommissioned away). Migrations have the -// pre-requisite that all required nodes are up and running so that we're able -// to execute all relevant node-level operations on them. If any of the nodes -// are found to be unavailable, an error is returned. -func (h *Helper) RequiredNodeIDs(ctx context.Context) ([]roachpb.NodeID, error) { - var nodeIDs []roachpb.NodeID - ls, err := h.nl.GetLivenessesFromKV(ctx) - if err != nil { - return nil, err - } - for _, l := range ls { - if l.Membership.Decommissioned() { - continue - } - live, err := h.nl.IsLive(l.NodeID) - if err != nil { - return nil, err - } - if !live { - return nil, errors.Newf("n%d required, but unavailable", l.NodeID) - } - nodeIDs = append(nodeIDs, l.NodeID) - } - return nodeIDs, nil -} - -// EveryNode invokes the given closure (named by the informational parameter op) -// across every node in the cluster. The mechanism for ensuring that we've done -// so, while accounting for the possibility of new nodes being added to the -// cluster in the interim, is provided by the following structure: -// (a) We'll retrieve the list of node IDs for all nodes in the system -// (b) For each node, we'll invoke the closure -// (c) We'll retrieve the list of node IDs again to account for the -// possibility of a new node being added during (b) -// (d) If there any discrepancies between the list retrieved in (a) -// and (c), we'll invoke the closure each node again -// (e) We'll continue to loop around until the node ID list stabilizes -// -// By the time EveryNode returns, we'll have thus invoked the closure against -// every node in the cluster. -// -// To consider one example of how this primitive is used, let's consider our use -// of it to bump the cluster version. After we return, given all nodes in the -// cluster will have their cluster versions bumped, and future node additions -// will observe the latest version (through the join RPC). This lets us author -// migrations that can assume that a certain version gate has been enabled on -// all nodes in the cluster, and will always be enabled for any new nodes in the -// system. -// -// It may be possible however that right after we return, a new node may join. -// This means that some migrations may have to be split up into two version -// bumps: one that phases out the old version (i.e. stops creation of stale data -// or behavior) and a clean-up version, which removes any vestiges of the stale -// data/behavior, and which, when active, ensures that the old data has vanished -// from the system. This is similar in spirit to how schema changes are split up -// into multiple smaller steps that are carried out sequentially. -func (h *Helper) EveryNode( - ctx context.Context, op string, fn func(context.Context, serverpb.MigrationClient) error, -) error { - nodeIDs, err := h.RequiredNodeIDs(ctx) - if err != nil { - return err - } - - for { - // TODO(irfansharif): We can/should send out these RPCs in parallel. - log.Infof(ctx, "executing op=%s on nodes=%s", op, nodeIDs) - for _, nodeID := range nodeIDs { - conn, err := h.dialer.Dial(ctx, nodeID, rpc.DefaultClass) - if err != nil { - return err - } - client := serverpb.NewMigrationClient(conn) - if err := fn(ctx, client); err != nil { - return err - } - } - - curNodeIDs, err := h.RequiredNodeIDs(ctx) - if err != nil { - return err - } - - if !identical(nodeIDs, curNodeIDs) { - nodeIDs = curNodeIDs - continue - } - - break - } - - return nil -} - -// nodeLiveness is the subset of the interface satisfied by CRDB's node liveness +// nodeLiveness is the subset of the interface satisfied by crdb's node liveness // component that the migration manager relies upon. type nodeLiveness interface { GetLivenessesFromKV(context.Context) ([]livenesspb.Liveness, error) @@ -180,7 +56,7 @@ type nodeLiveness interface { // // TODO(irfansharif): We'll need to eventually plumb in on a lease manager here. func NewManager( - dialer *nodedialer.Dialer, nl nodeLiveness, executor *sql.InternalExecutor, db *kv.DB, + dialer *nodedialer.Dialer, nl nodeLiveness, executor sqlutil.InternalExecutor, db *kv.DB, ) *Manager { return &Manager{ dialer: dialer, @@ -196,7 +72,7 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe // TODO(irfansharif): Should we inject every ctx here with specific labels // for each migration, so they log distinctly? ctx = logtags.AddTag(ctx, "migration-mgr", nil) - if from == to { + if from == to { // XXX: Stop gap for until I write real tests. // Nothing to do here. log.Infof(ctx, "no need to migrate, cluster already at newest version") return nil @@ -205,9 +81,6 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe // TODO(irfansharif): We'll need to acquire a lease here and refresh it // throughout during the migration to ensure mutual exclusion. - // TODO(irfansharif): We'll need to create a system table to store - // in-progress state of long running migrations, for introspection. - clusterVersions := clusterversion.ListBetween(from, to) if len(clusterVersions) == 0 { // We're attempt to migrate to something that's not defined in cluster @@ -220,10 +93,17 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe // here instead, so that we're able to actually migrate into it. clusterVersions = append(clusterVersions, to) } - log.Infof(ctx, "migrating cluster from %s to %s (stepping through %s)", from, to, clusterVersions) + log.Infof(ctx, "migrating cluster from %s to %s (stepping through %s)", + from, to, clusterVersions) for _, clusterVersion := range clusterVersions { - h := &Helper{Manager: m} + cluster := &clusterImpl{ + nl: m.nl, + dialer: m.dialer, + exec: m.executor, + kvDB: m.db, + } + h := newHelper(cluster, clusterVersion) // Push out the version gate to every node in the cluster. Each node // will persist the version, bump the local version gates, and then @@ -301,41 +181,53 @@ func (m *Manager) Migrate(ctx context.Context, from, to clusterversion.ClusterVe }); err != nil { return err } + } - // TODO(irfansharif): We'll want to retrieve the right migration off of - // our registry of migrations, if any, and execute it. // TODO(irfansharif): We'll want to be able to override which migration // is retrieved here within tests. We could make the registry be a part // of the manager, and all tests to provide their own. - _ = Registry[clusterVersion] - } - return nil -} + // Finally, run the actual migration. + migration, ok := registry[clusterVersion] + if !ok { + log.Infof(ctx, "no migration registered for %s, skipping", clusterVersion) + continue + } -// fenceVersionFor constructs the appropriate "fence version" for the given -// cluster version. Fence versions allow the migrations infrastructure to safely -// step through consecutive cluster versions in the presence of nodes (running -// any binary version) being added to the cluster. See the migration manager -// above for intended usage. -// -// Fence versions (and the migrations infrastructure entirely) were introduced -// in the 21.1 release cycle. In the same release cycle, we introduced the -// invariant that new user-defined versions (users being crdb engineers) must -// always have even-numbered Internal versions, thus reserving the odd numbers -// to slot in fence versions for each cluster version. See top-level -// documentation in pkg/clusterversion for more details. -func fenceVersionFor( - ctx context.Context, cv clusterversion.ClusterVersion, -) clusterversion.ClusterVersion { - if (cv.Internal % 2) != 0 { - log.Fatalf(ctx, "only even numbered internal versions allowed, found %s", cv.Version) + if err := migration.Run(ctx, h); err != nil { + return err + } + + { + // GC all gc-able replicas, which makes for a useful invariant for + // the migrations infrastructure to provide. For below-raft + // migrations, this ensures that there are no extant replicas that + // have not processed the corresponding Migrate command. + req := &serverpb.GCReplicasRequest{} + op := fmt.Sprint("gc-replicas") + if err := h.EveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { + _, err := client.GCReplicas(ctx, req) + return err + }); err != nil { + return err + } + } + { + // Make sure that all stores have synced, which makes for a useful + // invariant for the migrations infrastructure to provide. For + // below-raft migrations, this ensures that the applied state is + // flushed to disk disk. + req := &serverpb.FlushAllEnginesRequest{} + op := fmt.Sprint("flush-stores") + if err := h.EveryNode(ctx, op, func(ctx context.Context, client serverpb.MigrationClient) error { + _, err := client.FlushAllEngines(ctx, req) + return err + }); err != nil { + return err + } + } } - // We'll pick the odd internal version preceding the cluster version, - // slotting ourselves right before it. - fenceCV := cv - fenceCV.Internal-- - return fenceCV + return nil } diff --git a/pkg/migration/manager_test.go b/pkg/migration/manager_test.go new file mode 100644 index 000000000000..e0d261ba2d0f --- /dev/null +++ b/pkg/migration/manager_test.go @@ -0,0 +1,29 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package migration + +import "testing" + +func TestXXX(t *testing.T) { + // XXX: Things I want to test. + // + // ValidateClusterVersion + // cluster version gate has been pushed out on all nodes + // migration registry + // flushing all engines, gc-ing all replicas + + // migration status updates (migration.Run), progress, recording in + // system.migrations. + // testing that migrate has been run on all nodes + // Test individual migrations + // + // Will want to use test cluster for the whole thing. +} diff --git a/pkg/migration/migrations.go b/pkg/migration/migrations.go index dfae1ee82155..453fb18ef4d8 100644 --- a/pkg/migration/migrations.go +++ b/pkg/migration/migrations.go @@ -10,16 +10,146 @@ package migration -import "github.com/cockroachdb/cockroach/pkg/clusterversion" +import ( + "bytes" + "context" + "fmt" -// Registry defines the global mapping between a cluster version, and the + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" +) + +// registry defines the global mapping between a cluster version and the // associated migration. The migration is only executed after a cluster-wide // bump of the version gate. -var Registry = make(map[clusterversion.ClusterVersion]Migration) +var registry = make(map[clusterversion.ClusterVersion]Migration) func init() { - // TODO(irfansharif): We'll want to register individual migrations with - // specific internal cluster versions here. - // - // Registry[clusterversion.ByKey(clusterversion.VersionWhatever)] = WhateverMigration + register(clusterversion.TruncatedAndRangeAppliedStateMigration, TruncatedStateMigration, + "use unreplicated TruncatedState and RangeAppliedState for all ranges") +} + +// Status represents the status of any given migration. This is captured in the +// system.migrations table. +type Status string + +const ( + // StatusRunning is for migrations that are currently in progress. + StatusRunning Status = "running" + // StatusSucceeded is for jobs migrations that have successfully completed. + StatusSucceeded Status = "succeeded" + // StatusFailed is for migrations that have failed to execute. + StatusFailed Status = "failed" +) + +// Migration defines a program to be executed once every node in the cluster is +// (a) running a specific binary version, and (b) has completed all prior +// migrations. +// +// Each migration is associated with a specific internal cluster version and is +// idempotent in nature. When setting the cluster version (via `SET CLUSTER +// SETTING version`), the manager process determines the set of migrations +// needed to bridge the gap between the current active cluster version, and the +// target one. See [1] for where that happens. +// +// To introduce a migration, start by adding version key to pkg/clusterversion +// and introducing a corresponding internal cluster version for it. See [2] for +// more details. Following that, define a Migration in this package and add it +// to the registry. Be sure to key it in with the new cluster version we just +// added. During cluster upgrades, once the operator is able to set a cluster +// version setting that's past the version that was introduced (typically the +// major release version the migration was introduced in), the manager will +// execute the defined migration before letting the upgrade finalize. +// +// If the migration requires below-Raft level changes ([3] is one example), +// you'll need to add a version switch and the relevant KV-level migration in +// [4]. See IterateRangeDescriptors and the Migrate KV request for more details. +// +// [1]: `(*Manager).Migrate` +// [2]: pkg/clusterversion/cockroach_versions.go +// [3]: TruncatedStateMigration +// [4]: pkg/kv/kvserver/batch_eval/cmd_migrate.go +type Migration struct { + cv clusterversion.ClusterVersion + fn migrationFn + desc string +} + +type migrationFn func(context.Context, *Helper) error + +// Run kickstarts the actual migration process. It's responsible for recording +// the ongoing status of the migration in the `system.migrations` table. +func (m *Migration) Run(ctx context.Context, h *Helper) (err error) { + defer func() { + if err != nil { + // Mark ourselves as failed and record the last known error for + // visibility. + _ = h.updateStatus(ctx, StatusFailed) + _ = h.UpdateProgress(ctx, fmt.Sprintf("error during migration: %s", err)) + } + }() + + if err := h.insertMigrationRecord(ctx, m.desc); err != nil { + return err + } + + if err := m.fn(ctx, h); err != nil { + return err + } + + if err := h.updateStatus(ctx, StatusSucceeded); err != nil { + return err + } + + return nil +} + +// defaultBlockSize controls +// We pull this block size 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: +// +// block size == 1 ~ 2h 46m +// block size == 50 ~ 3m 20s +// block size == 200 ~ 50s +const defaultBlockSize = 200 + +func TruncatedStateMigration(ctx context.Context, h *Helper) error { + var batchIdx, numMigratedRanges int + init := func() { batchIdx, numMigratedRanges = 1, 0 } + if err := h.IterateRangeDescriptors(ctx, defaultBlockSize, init, func(descriptors ...roachpb.RangeDescriptor) error { + for _, desc := range descriptors { + // 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 + } + } + + numMigratedRanges += len(descriptors) + progress := fmt.Sprintf("[batch %d/??] migrated %d ranges", batchIdx, numMigratedRanges) + if err := h.UpdateProgress(ctx, progress); err != nil { + return err + } + batchIdx++ + + return nil + }); err != nil { + return err + } + + progress := fmt.Sprintf("[batch %d/%d] migrated %d ranges", batchIdx, batchIdx, numMigratedRanges) + if err := h.UpdateProgress(ctx, progress); err != nil { + return err + } + + return nil } diff --git a/pkg/migration/util.go b/pkg/migration/util.go index e638dff8d5f4..d821bc1f9f07 100644 --- a/pkg/migration/util.go +++ b/pkg/migration/util.go @@ -11,27 +11,111 @@ package migration import ( + "context" + "fmt" "sort" + "strings" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/redact" ) -// identical returns whether or not two lists of node IDs are identical as sets. -func identical(a, b []roachpb.NodeID) bool { +// node captures the relevant bits of each node as it pertains to the migration +// infrastructure. +type node struct { + id roachpb.NodeID + epoch int64 +} + +// nodes is a collection of node objects. +type nodes []node + +// identical returns whether or not two lists of nodes are identical as sets, +// and if not, what changed. The textual diff is only to be used for logging +// purposes. +func (ns nodes) identical(other nodes) (ok bool, diff string) { + a, b := ns, other if len(a) != len(b) { - return false + if len(a) < len(b) { + diff = fmt.Sprintf("node added to the cluster") + } else { + diff = fmt.Sprintf("node removed from the cluster") + } + return false, diff } + + // Sort by node IDs. sort.Slice(a, func(i, j int) bool { - return a[i] < a[j] + return a[i].id < a[j].id }) sort.Slice(b, func(i, j int) bool { - return b[i] < b[j] + return b[i].id < b[j].id }) - for i, v := range a { - if v != b[i] { - return false + for i := 0; i < len(a); i++ { + if a[i].id != b[i].id { + return false, "found different set of nodes" + } + if a[i].epoch != b[i].epoch { + return false, fmt.Sprintf("n%d was restarted", a[i].id) } } - return true + return true, "" +} + +func (ns nodes) String() string { + var b strings.Builder + b.WriteString("n{") + if len(ns) > 0 { + b.WriteString(fmt.Sprintf("%d", ns[0].id)) + for _, node := range ns[1:] { + b.WriteString(fmt.Sprintf(",%d", node.id)) + } + } + b.WriteString("}") + + return b.String() +} + +// SafeFormat implements redact.SafeFormatter. +func (ns nodes) SafeFormat(s redact.SafePrinter, _ rune) { + s.SafeString(redact.SafeString(ns.String())) +} + +// fenceVersionFor constructs the appropriate "fence version" for the given +// cluster version. Fence versions allow the migrations infrastructure to safely +// step through consecutive cluster versions in the presence of nodes (running +// any binary version) being added to the cluster. See the migration manager +// above for intended usage. +// +// Fence versions (and the migrations infrastructure entirely) were introduced +// in the 21.1 release cycle. In the same release cycle, we introduced the +// invariant that new user-defined versions (users being crdb engineers) must +// always have even-numbered Internal versions, thus reserving the odd numbers +// to slot in fence versions for each cluster version. See top-level +// documentation in pkg/clusterversion for more details. +func fenceVersionFor( + ctx context.Context, cv clusterversion.ClusterVersion, +) clusterversion.ClusterVersion { + if (cv.Internal % 2) != 0 { + log.Fatalf(ctx, "only even numbered internal versions allowed, found %s", cv.Version) + } + + // We'll pick the odd internal version preceding the cluster version, + // slotting ourselves right before it. + fenceCV := cv + fenceCV.Internal-- + return fenceCV +} + +// register is a short hand to register a given migration within the global +// registry. +func register(key clusterversion.Key, fn migrationFn, desc string) { + cv := clusterversion.ClusterVersion{Version: clusterversion.ByKey(key)} + if _, ok := registry[cv]; ok { + log.Fatalf(context.Background(), "doubly registering migration for %s", cv) + } + registry[cv] = Migration{cv: cv, fn: fn, desc: desc} } diff --git a/pkg/migration/util_test.go b/pkg/migration/util_test.go new file mode 100644 index 000000000000..55b001ebd85f --- /dev/null +++ b/pkg/migration/util_test.go @@ -0,0 +1,87 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package migration + +import ( + "strconv" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" +) + +func TestNodesString(t *testing.T) { + defer leaktest.AfterTest(t) + + ns := func(ids ...int) nodes { + var nodes []node + for _, id := range ids { + nodes = append(nodes, node{id: roachpb.NodeID(id)}) + } + return nodes + } + + var tests = []struct { + ns nodes + exp string + }{ + {ns(), "n{}"}, + {ns(1), "n{1}"}, + {ns(1, 2, 3), "n{1,2,3}"}, + {ns(3, 4, 7), "n{3,4,7}"}, + } + + for _, test := range tests { + if got := test.ns.String(); got != test.exp { + t.Fatalf("expected %s, got %s", test.exp, got) + } + } +} + +func TestNodesIdentical(t *testing.T) { + defer leaktest.AfterTest(t) + + list := func(nodes ...string) nodes { // takes in strings of the form "id@epoch" + var ns []node + for _, n := range nodes { + parts := strings.Split(n, "@") + id, _ := strconv.Atoi(parts[0]) + epoch, _ := strconv.Atoi(parts[1]) + ns = append(ns, node{id: roachpb.NodeID(id), epoch: int64(epoch)}) + } + return ns + } + + var tests = []struct { + a, b nodes + expOk bool + expDiff string + }{ + {list(), list(), true, ""}, + {list("1@2"), list("1@2"), true, ""}, + {list("1@2"), list("1@3"), false, "n1 was restarted"}, + {list("3@2"), list("5@2"), false, "found different set of nodes"}, + {list("2@1", "1@2"), list("1@2", "2@1"), true, ""}, + {list("1@2"), list("1@2", "2@1"), false, "node added to the cluster"}, + {list("1@1", "2@1"), list("1@1"), false, "node removed from the cluster"}, + } + + for _, test := range tests { + ok, diff := test.a.identical(test.b) + if ok != test.expOk { + t.Fatalf("expected identical = %t, got %t", test.expOk, ok) + } + if diff != test.expDiff { + t.Fatalf("expected diff %q, got %q", test.expDiff, diff) + } + } +} diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 63a3b374477c..bb34e21b52f3 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -708,6 +708,9 @@ func (*ImportRequest) Method() Method { return Import } // Method implements the Request interface. func (*AdminScatterRequest) Method() Method { return AdminScatter } +// Method implements the Request interface. +func (*MigrateRequest) Method() Method { return Migrate } + // Method implements the Request interface. func (*AddSSTableRequest) Method() Method { return AddSSTable } @@ -951,6 +954,12 @@ func (r *AdminScatterRequest) 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 *AddSSTableRequest) ShallowCopy() Request { shallowCopy := *r @@ -1283,6 +1292,7 @@ func (*ExportRequest) flags() int { return isRead | isRan func (*ImportRequest) flags() int { return isAdmin | isAlone } func (*AdminScatterRequest) flags() int { return isAdmin | isRange | isAlone } func (*AdminVerifyProtectedTimestampRequest) flags() int { return isAdmin | isRange | isAlone } +func (*MigrateRequest) flags() int { return isWrite | isRange | isAlone } func (*AddSSTableRequest) flags() int { return isWrite | isRange | isAlone | isUnsplittable | canBackpressure } diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 8806b751f919..128e9d9ef7d1 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -73,7 +73,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{0} + return fileDescriptor_api_2661ffa68e6772f8, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -101,7 +101,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{1} + return fileDescriptor_api_2661ffa68e6772f8, []int{1} } type ChecksumMode int32 @@ -148,7 +148,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{2} + return fileDescriptor_api_2661ffa68e6772f8, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -179,7 +179,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{3} + return fileDescriptor_api_2661ffa68e6772f8, []int{3} } type ExternalStorageProvider int32 @@ -220,7 +220,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{4} + return fileDescriptor_api_2661ffa68e6772f8, []int{4} } type MVCCFilter int32 @@ -243,7 +243,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{5} + return fileDescriptor_api_2661ffa68e6772f8, []int{5} } type ResponseHeader_ResumeReason int32 @@ -269,7 +269,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{1, 0} + return fileDescriptor_api_2661ffa68e6772f8, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -311,7 +311,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{25, 0} + return fileDescriptor_api_2661ffa68e6772f8, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -332,7 +332,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) String() string { return proto.CompactTextString(m) } func (*RequestHeader) ProtoMessage() {} func (*RequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{0} + return fileDescriptor_api_2661ffa68e6772f8, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -403,7 +403,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } func (*ResponseHeader) ProtoMessage() {} func (*ResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{1} + return fileDescriptor_api_2661ffa68e6772f8, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -437,7 +437,7 @@ func (m *GetRequest) Reset() { *m = GetRequest{} } func (m *GetRequest) String() string { return proto.CompactTextString(m) } func (*GetRequest) ProtoMessage() {} func (*GetRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{2} + return fileDescriptor_api_2661ffa68e6772f8, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -480,7 +480,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} } func (m *GetResponse) String() string { return proto.CompactTextString(m) } func (*GetResponse) ProtoMessage() {} func (*GetResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{3} + return fileDescriptor_api_2661ffa68e6772f8, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -523,7 +523,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} } func (m *PutRequest) String() string { return proto.CompactTextString(m) } func (*PutRequest) ProtoMessage() {} func (*PutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{4} + return fileDescriptor_api_2661ffa68e6772f8, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -557,7 +557,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} } func (m *PutResponse) String() string { return proto.CompactTextString(m) } func (*PutResponse) ProtoMessage() {} func (*PutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{5} + return fileDescriptor_api_2661ffa68e6772f8, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -639,7 +639,7 @@ func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} } func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) } func (*ConditionalPutRequest) ProtoMessage() {} func (*ConditionalPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{6} + return fileDescriptor_api_2661ffa68e6772f8, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -674,7 +674,7 @@ func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{} func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) } func (*ConditionalPutResponse) ProtoMessage() {} func (*ConditionalPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{7} + return fileDescriptor_api_2661ffa68e6772f8, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -720,7 +720,7 @@ func (m *InitPutRequest) Reset() { *m = InitPutRequest{} } func (m *InitPutRequest) String() string { return proto.CompactTextString(m) } func (*InitPutRequest) ProtoMessage() {} func (*InitPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{8} + return fileDescriptor_api_2661ffa68e6772f8, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -754,7 +754,7 @@ func (m *InitPutResponse) Reset() { *m = InitPutResponse{} } func (m *InitPutResponse) String() string { return proto.CompactTextString(m) } func (*InitPutResponse) ProtoMessage() {} func (*InitPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{9} + return fileDescriptor_api_2661ffa68e6772f8, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -794,7 +794,7 @@ func (m *IncrementRequest) Reset() { *m = IncrementRequest{} } func (m *IncrementRequest) String() string { return proto.CompactTextString(m) } func (*IncrementRequest) ProtoMessage() {} func (*IncrementRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{10} + return fileDescriptor_api_2661ffa68e6772f8, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -831,7 +831,7 @@ func (m *IncrementResponse) Reset() { *m = IncrementResponse{} } func (m *IncrementResponse) String() string { return proto.CompactTextString(m) } func (*IncrementResponse) ProtoMessage() {} func (*IncrementResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{11} + return fileDescriptor_api_2661ffa68e6772f8, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -865,7 +865,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRequest) ProtoMessage() {} func (*DeleteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{12} + return fileDescriptor_api_2661ffa68e6772f8, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -899,7 +899,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} } func (m *DeleteResponse) String() string { return proto.CompactTextString(m) } func (*DeleteResponse) ProtoMessage() {} func (*DeleteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{13} + return fileDescriptor_api_2661ffa68e6772f8, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -951,7 +951,7 @@ func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} } func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRangeRequest) ProtoMessage() {} func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{14} + return fileDescriptor_api_2661ffa68e6772f8, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -988,7 +988,7 @@ func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} } func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) } func (*DeleteRangeResponse) ProtoMessage() {} func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{15} + return fileDescriptor_api_2661ffa68e6772f8, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1043,7 +1043,7 @@ func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} } func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) } func (*ClearRangeRequest) ProtoMessage() {} func (*ClearRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{16} + return fileDescriptor_api_2661ffa68e6772f8, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1077,7 +1077,7 @@ func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} } func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) } func (*ClearRangeResponse) ProtoMessage() {} func (*ClearRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{17} + return fileDescriptor_api_2661ffa68e6772f8, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1118,7 +1118,7 @@ func (m *RevertRangeRequest) Reset() { *m = RevertRangeRequest{} } func (m *RevertRangeRequest) String() string { return proto.CompactTextString(m) } func (*RevertRangeRequest) ProtoMessage() {} func (*RevertRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{18} + return fileDescriptor_api_2661ffa68e6772f8, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1152,7 +1152,7 @@ func (m *RevertRangeResponse) Reset() { *m = RevertRangeResponse{} } func (m *RevertRangeResponse) String() string { return proto.CompactTextString(m) } func (*RevertRangeResponse) ProtoMessage() {} func (*RevertRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{19} + return fileDescriptor_api_2661ffa68e6772f8, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1203,7 +1203,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} } func (m *ScanRequest) String() string { return proto.CompactTextString(m) } func (*ScanRequest) ProtoMessage() {} func (*ScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{20} + return fileDescriptor_api_2661ffa68e6772f8, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1255,7 +1255,7 @@ func (m *ScanResponse) Reset() { *m = ScanResponse{} } func (m *ScanResponse) String() string { return proto.CompactTextString(m) } func (*ScanResponse) ProtoMessage() {} func (*ScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{21} + return fileDescriptor_api_2661ffa68e6772f8, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1306,7 +1306,7 @@ func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} } func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) } func (*ReverseScanRequest) ProtoMessage() {} func (*ReverseScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{22} + return fileDescriptor_api_2661ffa68e6772f8, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1358,7 +1358,7 @@ func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} } func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) } func (*ReverseScanResponse) ProtoMessage() {} func (*ReverseScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{23} + return fileDescriptor_api_2661ffa68e6772f8, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1411,7 +1411,7 @@ func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyRequest) ProtoMessage() {} func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{24} + return fileDescriptor_api_2661ffa68e6772f8, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1448,7 +1448,7 @@ func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyRespon func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse) ProtoMessage() {} func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{25} + return fileDescriptor_api_2661ffa68e6772f8, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1492,7 +1492,7 @@ func (m *CheckConsistencyResponse_Result) Reset() { *m = CheckConsistenc func (m *CheckConsistencyResponse_Result) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse_Result) ProtoMessage() {} func (*CheckConsistencyResponse_Result) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{25, 0} + return fileDescriptor_api_2661ffa68e6772f8, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1540,7 +1540,7 @@ func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} } func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsRequest) ProtoMessage() {} func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{26} + return fileDescriptor_api_2661ffa68e6772f8, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1576,7 +1576,7 @@ func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{} func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsResponse) ProtoMessage() {} func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{27} + return fileDescriptor_api_2661ffa68e6772f8, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1687,7 +1687,7 @@ func (m *EndTxnRequest) Reset() { *m = EndTxnRequest{} } func (m *EndTxnRequest) String() string { return proto.CompactTextString(m) } func (*EndTxnRequest) ProtoMessage() {} func (*EndTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{28} + return fileDescriptor_api_2661ffa68e6772f8, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1733,7 +1733,7 @@ func (m *EndTxnResponse) Reset() { *m = EndTxnResponse{} } func (m *EndTxnResponse) String() string { return proto.CompactTextString(m) } func (*EndTxnResponse) ProtoMessage() {} func (*EndTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{29} + return fileDescriptor_api_2661ffa68e6772f8, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1794,7 +1794,7 @@ func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} } func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminSplitRequest) ProtoMessage() {} func (*AdminSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{30} + return fileDescriptor_api_2661ffa68e6772f8, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1829,7 +1829,7 @@ func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} } func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminSplitResponse) ProtoMessage() {} func (*AdminSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{31} + return fileDescriptor_api_2661ffa68e6772f8, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1868,7 +1868,7 @@ func (m *AdminUnsplitRequest) Reset() { *m = AdminUnsplitRequest{} } func (m *AdminUnsplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitRequest) ProtoMessage() {} func (*AdminUnsplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{32} + return fileDescriptor_api_2661ffa68e6772f8, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1903,7 +1903,7 @@ func (m *AdminUnsplitResponse) Reset() { *m = AdminUnsplitResponse{} } func (m *AdminUnsplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitResponse) ProtoMessage() {} func (*AdminUnsplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{33} + return fileDescriptor_api_2661ffa68e6772f8, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1946,7 +1946,7 @@ func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} } func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) } func (*AdminMergeRequest) ProtoMessage() {} func (*AdminMergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{34} + return fileDescriptor_api_2661ffa68e6772f8, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1981,7 +1981,7 @@ func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} } func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) } func (*AdminMergeResponse) ProtoMessage() {} func (*AdminMergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{35} + return fileDescriptor_api_2661ffa68e6772f8, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2019,7 +2019,7 @@ func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseReq func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseRequest) ProtoMessage() {} func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{36} + return fileDescriptor_api_2661ffa68e6772f8, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2052,7 +2052,7 @@ func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseRe func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseResponse) ProtoMessage() {} func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{37} + return fileDescriptor_api_2661ffa68e6772f8, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2087,7 +2087,7 @@ func (m *ReplicationChange) Reset() { *m = ReplicationChange{} } func (m *ReplicationChange) String() string { return proto.CompactTextString(m) } func (*ReplicationChange) ProtoMessage() {} func (*ReplicationChange) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{38} + return fileDescriptor_api_2661ffa68e6772f8, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2145,7 +2145,7 @@ func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasR func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasRequest) ProtoMessage() {} func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{39} + return fileDescriptor_api_2661ffa68e6772f8, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2180,7 +2180,7 @@ func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicas func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasResponse) ProtoMessage() {} func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{40} + return fileDescriptor_api_2661ffa68e6772f8, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2217,7 +2217,7 @@ func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeReq func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeRequest) ProtoMessage() {} func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{41} + return fileDescriptor_api_2661ffa68e6772f8, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2250,7 +2250,7 @@ func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeRe func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeResponse) ProtoMessage() {} func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{42} + return fileDescriptor_api_2661ffa68e6772f8, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2289,7 +2289,7 @@ func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} } func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnRequest) ProtoMessage() {} func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{43} + return fileDescriptor_api_2661ffa68e6772f8, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2326,7 +2326,7 @@ func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} } func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnResponse) ProtoMessage() {} func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{44} + return fileDescriptor_api_2661ffa68e6772f8, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2364,7 +2364,7 @@ func (m *GCRequest) Reset() { *m = GCRequest{} } func (m *GCRequest) String() string { return proto.CompactTextString(m) } func (*GCRequest) ProtoMessage() {} func (*GCRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{45} + return fileDescriptor_api_2661ffa68e6772f8, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2398,7 +2398,7 @@ func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} } func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) } func (*GCRequest_GCKey) ProtoMessage() {} func (*GCRequest_GCKey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{45, 0} + return fileDescriptor_api_2661ffa68e6772f8, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2432,7 +2432,7 @@ func (m *GCResponse) Reset() { *m = GCResponse{} } func (m *GCResponse) String() string { return proto.CompactTextString(m) } func (*GCResponse) ProtoMessage() {} func (*GCResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{46} + return fileDescriptor_api_2661ffa68e6772f8, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2501,7 +2501,7 @@ func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} } func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) } func (*PushTxnRequest) ProtoMessage() {} func (*PushTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{47} + return fileDescriptor_api_2661ffa68e6772f8, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2544,7 +2544,7 @@ func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} } func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) } func (*PushTxnResponse) ProtoMessage() {} func (*PushTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{48} + return fileDescriptor_api_2661ffa68e6772f8, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2591,7 +2591,7 @@ func (m *RecoverTxnRequest) Reset() { *m = RecoverTxnRequest{} } func (m *RecoverTxnRequest) String() string { return proto.CompactTextString(m) } func (*RecoverTxnRequest) ProtoMessage() {} func (*RecoverTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{49} + return fileDescriptor_api_2661ffa68e6772f8, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2627,7 +2627,7 @@ func (m *RecoverTxnResponse) Reset() { *m = RecoverTxnResponse{} } func (m *RecoverTxnResponse) String() string { return proto.CompactTextString(m) } func (*RecoverTxnResponse) ProtoMessage() {} func (*RecoverTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{50} + return fileDescriptor_api_2661ffa68e6772f8, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2671,7 +2671,7 @@ func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} } func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) } func (*QueryTxnRequest) ProtoMessage() {} func (*QueryTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{51} + return fileDescriptor_api_2661ffa68e6772f8, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2715,7 +2715,7 @@ func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} } func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) } func (*QueryTxnResponse) ProtoMessage() {} func (*QueryTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{52} + return fileDescriptor_api_2661ffa68e6772f8, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2775,7 +2775,7 @@ func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} } func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) } func (*QueryIntentRequest) ProtoMessage() {} func (*QueryIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{53} + return fileDescriptor_api_2661ffa68e6772f8, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2811,7 +2811,7 @@ func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} } func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) } func (*QueryIntentResponse) ProtoMessage() {} func (*QueryIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{54} + return fileDescriptor_api_2661ffa68e6772f8, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2858,7 +2858,7 @@ func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} } func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRequest) ProtoMessage() {} func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{55} + return fileDescriptor_api_2661ffa68e6772f8, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2893,7 +2893,7 @@ func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} } func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentResponse) ProtoMessage() {} func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{56} + return fileDescriptor_api_2661ffa68e6772f8, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2944,7 +2944,7 @@ func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeReq func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeRequest) ProtoMessage() {} func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{57} + return fileDescriptor_api_2661ffa68e6772f8, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2979,7 +2979,7 @@ func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeRe func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeResponse) ProtoMessage() {} func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{58} + return fileDescriptor_api_2661ffa68e6772f8, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3016,7 +3016,7 @@ func (m *MergeRequest) Reset() { *m = MergeRequest{} } func (m *MergeRequest) String() string { return proto.CompactTextString(m) } func (*MergeRequest) ProtoMessage() {} func (*MergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{59} + return fileDescriptor_api_2661ffa68e6772f8, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3050,7 +3050,7 @@ func (m *MergeResponse) Reset() { *m = MergeResponse{} } func (m *MergeResponse) String() string { return proto.CompactTextString(m) } func (*MergeResponse) ProtoMessage() {} func (*MergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{60} + return fileDescriptor_api_2661ffa68e6772f8, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3095,7 +3095,7 @@ func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} } func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) } func (*TruncateLogRequest) ProtoMessage() {} func (*TruncateLogRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{61} + return fileDescriptor_api_2661ffa68e6772f8, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3129,7 +3129,7 @@ func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} } func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) } func (*TruncateLogResponse) ProtoMessage() {} func (*TruncateLogResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{62} + return fileDescriptor_api_2661ffa68e6772f8, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3173,7 +3173,7 @@ func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} } func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) } func (*RequestLeaseRequest) ProtoMessage() {} func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{63} + return fileDescriptor_api_2661ffa68e6772f8, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3222,7 +3222,7 @@ func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} } func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*TransferLeaseRequest) ProtoMessage() {} func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{64} + return fileDescriptor_api_2661ffa68e6772f8, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3259,7 +3259,7 @@ func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} } func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) } func (*LeaseInfoRequest) ProtoMessage() {} func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{65} + return fileDescriptor_api_2661ffa68e6772f8, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3296,7 +3296,7 @@ func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) } func (*LeaseInfoResponse) ProtoMessage() {} func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{66} + return fileDescriptor_api_2661ffa68e6772f8, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3331,7 +3331,7 @@ func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} } func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) } func (*RequestLeaseResponse) ProtoMessage() {} func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{67} + return fileDescriptor_api_2661ffa68e6772f8, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3386,7 +3386,7 @@ func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{} func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumRequest) ProtoMessage() {} func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{68} + return fileDescriptor_api_2661ffa68e6772f8, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3423,7 +3423,7 @@ func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumResponse) ProtoMessage() {} func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{69} + return fileDescriptor_api_2661ffa68e6772f8, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3463,7 +3463,7 @@ func (m *ExternalStorage) Reset() { *m = ExternalStorage{} } func (m *ExternalStorage) String() string { return proto.CompactTextString(m) } func (*ExternalStorage) ProtoMessage() {} func (*ExternalStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70} + return fileDescriptor_api_2661ffa68e6772f8, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3497,7 +3497,7 @@ func (m *ExternalStorage_LocalFilePath) Reset() { *m = ExternalStorage_L func (m *ExternalStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_LocalFilePath) ProtoMessage() {} func (*ExternalStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70, 0} + return fileDescriptor_api_2661ffa68e6772f8, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3530,7 +3530,7 @@ func (m *ExternalStorage_Http) Reset() { *m = ExternalStorage_Http{} } func (m *ExternalStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Http) ProtoMessage() {} func (*ExternalStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70, 1} + return fileDescriptor_api_2661ffa68e6772f8, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3572,7 +3572,7 @@ func (m *ExternalStorage_S3) Reset() { *m = ExternalStorage_S3{} } func (m *ExternalStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_S3) ProtoMessage() {} func (*ExternalStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70, 2} + return fileDescriptor_api_2661ffa68e6772f8, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3611,7 +3611,7 @@ func (m *ExternalStorage_GCS) Reset() { *m = ExternalStorage_GCS{} } func (m *ExternalStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_GCS) ProtoMessage() {} func (*ExternalStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70, 3} + return fileDescriptor_api_2661ffa68e6772f8, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3647,7 +3647,7 @@ func (m *ExternalStorage_Azure) Reset() { *m = ExternalStorage_Azure{} } func (m *ExternalStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Azure) ProtoMessage() {} func (*ExternalStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70, 4} + return fileDescriptor_api_2661ffa68e6772f8, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3686,7 +3686,7 @@ func (m *ExternalStorage_Workload) Reset() { *m = ExternalStorage_Worklo func (m *ExternalStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Workload) ProtoMessage() {} func (*ExternalStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70, 5} + return fileDescriptor_api_2661ffa68e6772f8, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3728,7 +3728,7 @@ func (m *ExternalStorage_FileTable) Reset() { *m = ExternalStorage_FileT func (m *ExternalStorage_FileTable) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_FileTable) ProtoMessage() {} func (*ExternalStorage_FileTable) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{70, 6} + return fileDescriptor_api_2661ffa68e6772f8, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3768,7 +3768,7 @@ func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} } func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) } func (*WriteBatchRequest) ProtoMessage() {} func (*WriteBatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{71} + return fileDescriptor_api_2661ffa68e6772f8, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3802,7 +3802,7 @@ func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} } func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) } func (*WriteBatchResponse) ProtoMessage() {} func (*WriteBatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{72} + return fileDescriptor_api_2661ffa68e6772f8, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3838,7 +3838,7 @@ func (m *FileEncryptionOptions) Reset() { *m = FileEncryptionOptions{} } func (m *FileEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*FileEncryptionOptions) ProtoMessage() {} func (*FileEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{73} + return fileDescriptor_api_2661ffa68e6772f8, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3908,7 +3908,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} } func (m *ExportRequest) String() string { return proto.CompactTextString(m) } func (*ExportRequest) ProtoMessage() {} func (*ExportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{74} + return fileDescriptor_api_2661ffa68e6772f8, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3960,7 +3960,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } func (*BulkOpSummary) ProtoMessage() {} func (*BulkOpSummary) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{75} + return fileDescriptor_api_2661ffa68e6772f8, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3996,7 +3996,7 @@ func (m *ExportResponse) Reset() { *m = ExportResponse{} } func (m *ExportResponse) String() string { return proto.CompactTextString(m) } func (*ExportResponse) ProtoMessage() {} func (*ExportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{76} + return fileDescriptor_api_2661ffa68e6772f8, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4036,7 +4036,7 @@ func (m *ExportResponse_File) Reset() { *m = ExportResponse_File{} } func (m *ExportResponse_File) String() string { return proto.CompactTextString(m) } func (*ExportResponse_File) ProtoMessage() {} func (*ExportResponse_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{76, 0} + return fileDescriptor_api_2661ffa68e6772f8, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4087,7 +4087,7 @@ func (m *ImportRequest) Reset() { *m = ImportRequest{} } func (m *ImportRequest) String() string { return proto.CompactTextString(m) } func (*ImportRequest) ProtoMessage() {} func (*ImportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{77} + return fileDescriptor_api_2661ffa68e6772f8, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4122,7 +4122,7 @@ func (m *ImportRequest_File) Reset() { *m = ImportRequest_File{} } func (m *ImportRequest_File) String() string { return proto.CompactTextString(m) } func (*ImportRequest_File) ProtoMessage() {} func (*ImportRequest_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{77, 0} + return fileDescriptor_api_2661ffa68e6772f8, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4158,7 +4158,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } func (*ImportRequest_TableRekey) ProtoMessage() {} func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{77, 1} + return fileDescriptor_api_2661ffa68e6772f8, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4193,7 +4193,7 @@ func (m *ImportResponse) Reset() { *m = ImportResponse{} } func (m *ImportResponse) String() string { return proto.CompactTextString(m) } func (*ImportResponse) ProtoMessage() {} func (*ImportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{78} + return fileDescriptor_api_2661ffa68e6772f8, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4231,7 +4231,7 @@ func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} } func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) } func (*AdminScatterRequest) ProtoMessage() {} func (*AdminScatterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{79} + return fileDescriptor_api_2661ffa68e6772f8, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4268,7 +4268,7 @@ func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} } func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse) ProtoMessage() {} func (*AdminScatterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{80} + return fileDescriptor_api_2661ffa68e6772f8, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4301,7 +4301,7 @@ func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse_Range) ProtoMessage() {} func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{80, 0} + return fileDescriptor_api_2661ffa68e6772f8, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4326,6 +4326,83 @@ func (m *AdminScatterResponse_Range) XXX_DiscardUnknown() { var xxx_messageInfo_AdminScatterResponse_Range 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_2661ffa68e6772f8, []int{81} +} +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_2661ffa68e6772f8, []int{82} +} +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 + // AdminVerifyProtectedTimestampRequest is the argument to the // AdminVerifyProtectedTimestamp method which ensures that the specified record // will be seen before data can be garbage collected at the timestamp. @@ -4346,7 +4423,7 @@ func (m *AdminVerifyProtectedTimestampRequest) Reset() { *m = AdminVerif func (m *AdminVerifyProtectedTimestampRequest) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampRequest) ProtoMessage() {} func (*AdminVerifyProtectedTimestampRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{81} + return fileDescriptor_api_2661ffa68e6772f8, []int{83} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4384,7 +4461,7 @@ func (m *AdminVerifyProtectedTimestampResponse) Reset() { *m = AdminVeri func (m *AdminVerifyProtectedTimestampResponse) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampResponse) ProtoMessage() {} func (*AdminVerifyProtectedTimestampResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{82} + return fileDescriptor_api_2661ffa68e6772f8, []int{84} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4437,7 +4514,7 @@ func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) } func (*AddSSTableRequest) ProtoMessage() {} func (*AddSSTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{83} + return fileDescriptor_api_2661ffa68e6772f8, []int{85} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4471,7 +4548,7 @@ func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} } func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) } func (*AddSSTableResponse) ProtoMessage() {} func (*AddSSTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{84} + return fileDescriptor_api_2661ffa68e6772f8, []int{86} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4515,7 +4592,7 @@ func (m *RefreshRequest) Reset() { *m = RefreshRequest{} } func (m *RefreshRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRequest) ProtoMessage() {} func (*RefreshRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{85} + return fileDescriptor_api_2661ffa68e6772f8, []int{87} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4549,7 +4626,7 @@ func (m *RefreshResponse) Reset() { *m = RefreshResponse{} } func (m *RefreshResponse) String() string { return proto.CompactTextString(m) } func (*RefreshResponse) ProtoMessage() {} func (*RefreshResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{86} + return fileDescriptor_api_2661ffa68e6772f8, []int{88} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4588,7 +4665,7 @@ func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} } func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRangeRequest) ProtoMessage() {} func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{87} + return fileDescriptor_api_2661ffa68e6772f8, []int{89} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4622,7 +4699,7 @@ func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} } func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) } func (*RefreshRangeResponse) ProtoMessage() {} func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{88} + return fileDescriptor_api_2661ffa68e6772f8, []int{90} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4671,7 +4748,7 @@ func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} } func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) } func (*SubsumeRequest) ProtoMessage() {} func (*SubsumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{89} + return fileDescriptor_api_2661ffa68e6772f8, []int{91} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4720,7 +4797,7 @@ func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } func (*SubsumeResponse) ProtoMessage() {} func (*SubsumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{90} + return fileDescriptor_api_2661ffa68e6772f8, []int{92} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4755,7 +4832,7 @@ func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} } func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RangeStatsRequest) ProtoMessage() {} func (*RangeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{91} + return fileDescriptor_api_2661ffa68e6772f8, []int{93} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4797,7 +4874,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RangeStatsResponse) ProtoMessage() {} func (*RangeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{92} + return fileDescriptor_api_2661ffa68e6772f8, []int{94} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4873,6 +4950,7 @@ type RequestUnion struct { // *RequestUnion_Subsume // *RequestUnion_RangeStats // *RequestUnion_AdminVerifyProtectedTimestamp + // *RequestUnion_Migrate Value isRequestUnion_Value `protobuf_oneof:"value"` } @@ -4880,7 +4958,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} } func (m *RequestUnion) String() string { return proto.CompactTextString(m) } func (*RequestUnion) ProtoMessage() {} func (*RequestUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{93} + return fileDescriptor_api_2661ffa68e6772f8, []int{95} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5043,6 +5121,9 @@ type RequestUnion_RangeStats struct { type RequestUnion_AdminVerifyProtectedTimestamp struct { AdminVerifyProtectedTimestamp *AdminVerifyProtectedTimestampRequest `protobuf:"bytes,49,opt,name=admin_verify_protected_timestamp,json=adminVerifyProtectedTimestamp,proto3,oneof"` } +type RequestUnion_Migrate struct { + Migrate *MigrateRequest `protobuf:"bytes,50,opt,name=migrate,proto3,oneof"` +} func (*RequestUnion_Get) isRequestUnion_Value() {} func (*RequestUnion_Put) isRequestUnion_Value() {} @@ -5088,6 +5169,7 @@ func (*RequestUnion_RefreshRange) isRequestUnion_Value() {} func (*RequestUnion_Subsume) isRequestUnion_Value() {} func (*RequestUnion_RangeStats) isRequestUnion_Value() {} func (*RequestUnion_AdminVerifyProtectedTimestamp) isRequestUnion_Value() {} +func (*RequestUnion_Migrate) isRequestUnion_Value() {} func (m *RequestUnion) GetValue() isRequestUnion_Value { if m != nil { @@ -5404,6 +5486,13 @@ func (m *RequestUnion) GetAdminVerifyProtectedTimestamp() *AdminVerifyProtectedT return nil } +func (m *RequestUnion) GetMigrate() *MigrateRequest { + if x, ok := m.GetValue().(*RequestUnion_Migrate); ok { + return x.Migrate + } + return nil +} + // XXX_OneofFuncs is for the internal use of the proto package. func (*RequestUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { return _RequestUnion_OneofMarshaler, _RequestUnion_OneofUnmarshaler, _RequestUnion_OneofSizer, []interface{}{ @@ -5451,6 +5540,7 @@ func (*RequestUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) (*RequestUnion_Subsume)(nil), (*RequestUnion_RangeStats)(nil), (*RequestUnion_AdminVerifyProtectedTimestamp)(nil), + (*RequestUnion_Migrate)(nil), } } @@ -5678,6 +5768,11 @@ func _RequestUnion_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { if err := b.EncodeMessage(x.AdminVerifyProtectedTimestamp); err != nil { return err } + case *RequestUnion_Migrate: + _ = b.EncodeVarint(50<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Migrate); err != nil { + return err + } case nil: default: return fmt.Errorf("RequestUnion.Value has unexpected type %T", x) @@ -6040,6 +6135,14 @@ func _RequestUnion_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.B err := b.DecodeMessage(msg) m.Value = &RequestUnion_AdminVerifyProtectedTimestamp{msg} return true, err + case 50: // value.migrate + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(MigrateRequest) + err := b.DecodeMessage(msg) + m.Value = &RequestUnion_Migrate{msg} + return true, err default: return false, nil } @@ -6269,6 +6372,11 @@ func _RequestUnion_OneofSizer(msg proto.Message) (n int) { n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s + case *RequestUnion_Migrate: + s := proto.Size(x.Migrate) + n += 2 // tag and wire + n += proto.SizeVarint(uint64(s)) + n += s case nil: default: panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) @@ -6323,6 +6431,7 @@ type ResponseUnion struct { // *ResponseUnion_Subsume // *ResponseUnion_RangeStats // *ResponseUnion_AdminVerifyProtectedTimestamp + // *ResponseUnion_Migrate Value isResponseUnion_Value `protobuf_oneof:"value"` } @@ -6330,7 +6439,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } func (*ResponseUnion) ProtoMessage() {} func (*ResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{94} + return fileDescriptor_api_2661ffa68e6772f8, []int{96} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6490,6 +6599,9 @@ type ResponseUnion_RangeStats struct { type ResponseUnion_AdminVerifyProtectedTimestamp struct { AdminVerifyProtectedTimestamp *AdminVerifyProtectedTimestampResponse `protobuf:"bytes,49,opt,name=admin_verify_protected_timestamp,json=adminVerifyProtectedTimestamp,proto3,oneof"` } +type ResponseUnion_Migrate struct { + Migrate *MigrateResponse `protobuf:"bytes,50,opt,name=migrate,proto3,oneof"` +} func (*ResponseUnion_Get) isResponseUnion_Value() {} func (*ResponseUnion_Put) isResponseUnion_Value() {} @@ -6534,6 +6646,7 @@ func (*ResponseUnion_RefreshRange) isResponseUnion_Value() {} func (*ResponseUnion_Subsume) isResponseUnion_Value() {} func (*ResponseUnion_RangeStats) isResponseUnion_Value() {} func (*ResponseUnion_AdminVerifyProtectedTimestamp) isResponseUnion_Value() {} +func (*ResponseUnion_Migrate) isResponseUnion_Value() {} func (m *ResponseUnion) GetValue() isResponseUnion_Value { if m != nil { @@ -6843,6 +6956,13 @@ func (m *ResponseUnion) GetAdminVerifyProtectedTimestamp() *AdminVerifyProtected return nil } +func (m *ResponseUnion) GetMigrate() *MigrateResponse { + if x, ok := m.GetValue().(*ResponseUnion_Migrate); ok { + return x.Migrate + } + return nil +} + // XXX_OneofFuncs is for the internal use of the proto package. func (*ResponseUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) { return _ResponseUnion_OneofMarshaler, _ResponseUnion_OneofUnmarshaler, _ResponseUnion_OneofSizer, []interface{}{ @@ -6889,6 +7009,7 @@ func (*ResponseUnion) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) (*ResponseUnion_Subsume)(nil), (*ResponseUnion_RangeStats)(nil), (*ResponseUnion_AdminVerifyProtectedTimestamp)(nil), + (*ResponseUnion_Migrate)(nil), } } @@ -7111,6 +7232,11 @@ func _ResponseUnion_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { if err := b.EncodeMessage(x.AdminVerifyProtectedTimestamp); err != nil { return err } + case *ResponseUnion_Migrate: + _ = b.EncodeVarint(50<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Migrate); err != nil { + return err + } case nil: default: return fmt.Errorf("ResponseUnion.Value has unexpected type %T", x) @@ -7465,6 +7591,14 @@ func _ResponseUnion_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto. err := b.DecodeMessage(msg) m.Value = &ResponseUnion_AdminVerifyProtectedTimestamp{msg} return true, err + case 50: // value.migrate + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(MigrateResponse) + err := b.DecodeMessage(msg) + m.Value = &ResponseUnion_Migrate{msg} + return true, err default: return false, nil } @@ -7689,6 +7823,11 @@ func _ResponseUnion_OneofSizer(msg proto.Message) (n int) { n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s + case *ResponseUnion_Migrate: + s := proto.Size(x.Migrate) + n += 2 // tag and wire + n += proto.SizeVarint(uint64(s)) + n += s case nil: default: panic(fmt.Sprintf("proto: unexpected type %T in oneof", x)) @@ -7865,7 +8004,7 @@ func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} func (*Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{95} + return fileDescriptor_api_2661ffa68e6772f8, []int{97} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7903,7 +8042,7 @@ func (m *ClientRangeInfo) Reset() { *m = ClientRangeInfo{} } func (m *ClientRangeInfo) String() string { return proto.CompactTextString(m) } func (*ClientRangeInfo) ProtoMessage() {} func (*ClientRangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{96} + return fileDescriptor_api_2661ffa68e6772f8, []int{98} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7939,7 +8078,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{97} + return fileDescriptor_api_2661ffa68e6772f8, []int{99} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7976,7 +8115,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{98} + return fileDescriptor_api_2661ffa68e6772f8, []int{100} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8055,7 +8194,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} } func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) } func (*BatchResponse_Header) ProtoMessage() {} func (*BatchResponse_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{98, 0} + return fileDescriptor_api_2661ffa68e6772f8, []int{100, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8093,7 +8232,7 @@ func (m *RangeLookupRequest) Reset() { *m = RangeLookupRequest{} } func (m *RangeLookupRequest) String() string { return proto.CompactTextString(m) } func (*RangeLookupRequest) ProtoMessage() {} func (*RangeLookupRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{99} + return fileDescriptor_api_2661ffa68e6772f8, []int{101} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8131,7 +8270,7 @@ func (m *RangeLookupResponse) Reset() { *m = RangeLookupResponse{} } func (m *RangeLookupResponse) String() string { return proto.CompactTextString(m) } func (*RangeLookupResponse) ProtoMessage() {} func (*RangeLookupResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{100} + return fileDescriptor_api_2661ffa68e6772f8, []int{102} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8170,7 +8309,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} func (*RangeFeedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{101} + return fileDescriptor_api_2661ffa68e6772f8, []int{103} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8211,7 +8350,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} func (*RangeFeedValue) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{102} + return fileDescriptor_api_2661ffa68e6772f8, []int{104} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8252,7 +8391,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{103} + return fileDescriptor_api_2661ffa68e6772f8, []int{105} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8289,7 +8428,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} func (*RangeFeedError) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{104} + return fileDescriptor_api_2661ffa68e6772f8, []int{106} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8326,7 +8465,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} func (*RangeFeedEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{105} + return fileDescriptor_api_2661ffa68e6772f8, []int{107} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8370,7 +8509,7 @@ func (m *ResetQuorumRequest) Reset() { *m = ResetQuorumRequest{} } func (m *ResetQuorumRequest) String() string { return proto.CompactTextString(m) } func (*ResetQuorumRequest) ProtoMessage() {} func (*ResetQuorumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{106} + return fileDescriptor_api_2661ffa68e6772f8, []int{108} } func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8402,7 +8541,7 @@ func (m *ResetQuorumResponse) Reset() { *m = ResetQuorumResponse{} } func (m *ResetQuorumResponse) String() string { return proto.CompactTextString(m) } func (*ResetQuorumResponse) ProtoMessage() {} func (*ResetQuorumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{107} + return fileDescriptor_api_2661ffa68e6772f8, []int{109} } func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8443,7 +8582,7 @@ func (m *GossipSubscriptionRequest) Reset() { *m = GossipSubscriptionReq func (m *GossipSubscriptionRequest) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionRequest) ProtoMessage() {} func (*GossipSubscriptionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{108} + return fileDescriptor_api_2661ffa68e6772f8, []int{110} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8483,7 +8622,7 @@ func (m *GossipSubscriptionEvent) Reset() { *m = GossipSubscriptionEvent func (m *GossipSubscriptionEvent) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionEvent) ProtoMessage() {} func (*GossipSubscriptionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{109} + return fileDescriptor_api_2661ffa68e6772f8, []int{111} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8519,7 +8658,7 @@ func (m *JoinNodeRequest) Reset() { *m = JoinNodeRequest{} } func (m *JoinNodeRequest) String() string { return proto.CompactTextString(m) } func (*JoinNodeRequest) ProtoMessage() {} func (*JoinNodeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{110} + return fileDescriptor_api_2661ffa68e6772f8, []int{112} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8558,7 +8697,7 @@ func (m *JoinNodeResponse) Reset() { *m = JoinNodeResponse{} } func (m *JoinNodeResponse) String() string { return proto.CompactTextString(m) } func (*JoinNodeResponse) ProtoMessage() {} func (*JoinNodeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{111} + return fileDescriptor_api_2661ffa68e6772f8, []int{113} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8601,7 +8740,7 @@ func (m *ContentionEvent) Reset() { *m = ContentionEvent{} } func (m *ContentionEvent) String() string { return proto.CompactTextString(m) } func (*ContentionEvent) ProtoMessage() {} func (*ContentionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{112} + return fileDescriptor_api_2661ffa68e6772f8, []int{114} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8723,6 +8862,8 @@ func init() { proto.RegisterType((*AdminScatterRequest)(nil), "cockroach.roachpb.AdminScatterRequest") proto.RegisterType((*AdminScatterResponse)(nil), "cockroach.roachpb.AdminScatterResponse") proto.RegisterType((*AdminScatterResponse_Range)(nil), "cockroach.roachpb.AdminScatterResponse.Range") + proto.RegisterType((*MigrateRequest)(nil), "cockroach.roachpb.MigrateRequest") + proto.RegisterType((*MigrateResponse)(nil), "cockroach.roachpb.MigrateResponse") proto.RegisterType((*AdminVerifyProtectedTimestampRequest)(nil), "cockroach.roachpb.AdminVerifyProtectedTimestampRequest") proto.RegisterType((*AdminVerifyProtectedTimestampResponse)(nil), "cockroach.roachpb.AdminVerifyProtectedTimestampResponse") proto.RegisterType((*AddSSTableRequest)(nil), "cockroach.roachpb.AddSSTableRequest") @@ -12979,7 +13120,7 @@ func (m *AdminScatterResponse_Range) MarshalTo(dAtA []byte) (int, error) { return i, nil } -func (m *AdminVerifyProtectedTimestampRequest) Marshal() (dAtA []byte, err error) { +func (m *MigrateRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) n, err := m.MarshalTo(dAtA) @@ -12989,7 +13130,7 @@ func (m *AdminVerifyProtectedTimestampRequest) Marshal() (dAtA []byte, err error return dAtA[:n], nil } -func (m *AdminVerifyProtectedTimestampRequest) MarshalTo(dAtA []byte) (int, error) { +func (m *MigrateRequest) MarshalTo(dAtA []byte) (int, error) { var i int _ = i var l int @@ -13004,28 +13145,88 @@ func (m *AdminVerifyProtectedTimestampRequest) MarshalTo(dAtA []byte) (int, erro i += n138 dAtA[i] = 0x12 i++ - i = encodeVarintApi(dAtA, i, uint64(m.Protected.Size())) - n139, err := m.Protected.MarshalTo(dAtA[i:]) + i = encodeVarintApi(dAtA, i, uint64(m.Version.Size())) + n139, err := m.Version.MarshalTo(dAtA[i:]) if err != nil { return 0, err } i += n139 + 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())) + n140, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n140 + return i, nil +} + +func (m *AdminVerifyProtectedTimestampRequest) 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 *AdminVerifyProtectedTimestampRequest) 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())) + n141, err := m.RequestHeader.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n141 + dAtA[i] = 0x12 + i++ + i = encodeVarintApi(dAtA, i, uint64(m.Protected.Size())) + n142, err := m.Protected.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n142 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.RecordAliveAt.Size())) - n140, err := m.RecordAliveAt.MarshalTo(dAtA[i:]) + n143, err := m.RecordAliveAt.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n140 + i += n143 dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecordID.Size())) - n141, err := m.RecordID.MarshalTo(dAtA[i:]) + n144, err := m.RecordID.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n141 + i += n144 return i, nil } @@ -13047,11 +13248,11 @@ func (m *AdminVerifyProtectedTimestampResponse) MarshalTo(dAtA []byte) (int, err dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n142, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n145, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n142 + i += n145 if m.Verified { dAtA[i] = 0x10 i++ @@ -13095,11 +13296,11 @@ func (m *AddSSTableRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n143, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n146, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n143 + i += n146 if len(m.Data) > 0 { dAtA[i] = 0x12 i++ @@ -13120,11 +13321,11 @@ func (m *AddSSTableRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.MVCCStats.Size())) - n144, err := m.MVCCStats.MarshalTo(dAtA[i:]) + n147, err := m.MVCCStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n144 + i += n147 } if m.IngestAsWrites { dAtA[i] = 0x28 @@ -13157,11 +13358,11 @@ func (m *AddSSTableResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n145, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n148, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n145 + i += n148 return i, nil } @@ -13183,19 +13384,19 @@ func (m *RefreshRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n146, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n149, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n146 + i += n149 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.RefreshFrom.Size())) - n147, err := m.RefreshFrom.MarshalTo(dAtA[i:]) + n150, err := m.RefreshFrom.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n147 + i += n150 return i, nil } @@ -13217,11 +13418,11 @@ func (m *RefreshResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n148, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n151, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n148 + i += n151 return i, nil } @@ -13243,19 +13444,19 @@ func (m *RefreshRangeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n149, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n152, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n149 + i += n152 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.RefreshFrom.Size())) - n150, err := m.RefreshFrom.MarshalTo(dAtA[i:]) + n153, err := m.RefreshFrom.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n150 + i += n153 return i, nil } @@ -13277,11 +13478,11 @@ func (m *RefreshRangeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n151, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n154, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n151 + i += n154 return i, nil } @@ -13303,27 +13504,27 @@ func (m *SubsumeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n152, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n155, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n152 + i += n155 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeftDesc.Size())) - n153, err := m.LeftDesc.MarshalTo(dAtA[i:]) + n156, err := m.LeftDesc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n153 + i += n156 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.RightDesc.Size())) - n154, err := m.RightDesc.MarshalTo(dAtA[i:]) + n157, err := m.RightDesc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n154 + i += n157 return i, nil } @@ -13345,19 +13546,19 @@ func (m *SubsumeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n155, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n158, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n155 + i += n158 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.MVCCStats.Size())) - n156, err := m.MVCCStats.MarshalTo(dAtA[i:]) + n159, err := m.MVCCStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n156 + i += n159 if m.LeaseAppliedIndex != 0 { dAtA[i] = 0x20 i++ @@ -13366,11 +13567,11 @@ func (m *SubsumeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.FreezeStart.Size())) - n157, err := m.FreezeStart.MarshalTo(dAtA[i:]) + n160, err := m.FreezeStart.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n157 + i += n160 return i, nil } @@ -13392,11 +13593,11 @@ func (m *RangeStatsRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestHeader.Size())) - n158, err := m.RequestHeader.MarshalTo(dAtA[i:]) + n161, err := m.RequestHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n158 + i += n161 return i, nil } @@ -13418,19 +13619,19 @@ func (m *RangeStatsResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.ResponseHeader.Size())) - n159, err := m.ResponseHeader.MarshalTo(dAtA[i:]) + n162, err := m.ResponseHeader.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n159 + i += n162 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.MVCCStats.Size())) - n160, err := m.MVCCStats.MarshalTo(dAtA[i:]) + n163, err := m.MVCCStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n160 + i += n163 if m.QueriesPerSecond != 0 { dAtA[i] = 0x19 i++ @@ -13441,11 +13642,11 @@ func (m *RangeStatsResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeInfo.Size())) - n161, err := m.RangeInfo.MarshalTo(dAtA[i:]) + n164, err := m.RangeInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n161 + i += n164 } return i, nil } @@ -13466,11 +13667,11 @@ func (m *RequestUnion) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Value != nil { - nn162, err := m.Value.MarshalTo(dAtA[i:]) + nn165, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn162 + i += nn165 } return i, nil } @@ -13481,11 +13682,11 @@ func (m *RequestUnion_Get) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Get.Size())) - n163, err := m.Get.MarshalTo(dAtA[i:]) + n166, err := m.Get.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n163 + i += n166 } return i, nil } @@ -13495,11 +13696,11 @@ func (m *RequestUnion_Put) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Put.Size())) - n164, err := m.Put.MarshalTo(dAtA[i:]) + n167, err := m.Put.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n164 + i += n167 } return i, nil } @@ -13509,11 +13710,11 @@ func (m *RequestUnion_ConditionalPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size())) - n165, err := m.ConditionalPut.MarshalTo(dAtA[i:]) + n168, err := m.ConditionalPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n165 + i += n168 } return i, nil } @@ -13523,11 +13724,11 @@ func (m *RequestUnion_Increment) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size())) - n166, err := m.Increment.MarshalTo(dAtA[i:]) + n169, err := m.Increment.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n166 + i += n169 } return i, nil } @@ -13537,11 +13738,11 @@ func (m *RequestUnion_Delete) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size())) - n167, err := m.Delete.MarshalTo(dAtA[i:]) + n170, err := m.Delete.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n167 + i += n170 } return i, nil } @@ -13551,11 +13752,11 @@ func (m *RequestUnion_DeleteRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size())) - n168, err := m.DeleteRange.MarshalTo(dAtA[i:]) + n171, err := m.DeleteRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n168 + i += n171 } return i, nil } @@ -13565,11 +13766,11 @@ func (m *RequestUnion_Scan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size())) - n169, err := m.Scan.MarshalTo(dAtA[i:]) + n172, err := m.Scan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n169 + i += n172 } return i, nil } @@ -13579,11 +13780,11 @@ func (m *RequestUnion_EndTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTxn.Size())) - n170, err := m.EndTxn.MarshalTo(dAtA[i:]) + n173, err := m.EndTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n170 + i += n173 } return i, nil } @@ -13593,11 +13794,11 @@ func (m *RequestUnion_AdminSplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size())) - n171, err := m.AdminSplit.MarshalTo(dAtA[i:]) + n174, err := m.AdminSplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n171 + i += n174 } return i, nil } @@ -13607,11 +13808,11 @@ func (m *RequestUnion_AdminMerge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size())) - n172, err := m.AdminMerge.MarshalTo(dAtA[i:]) + n175, err := m.AdminMerge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n172 + i += n175 } return i, nil } @@ -13621,11 +13822,11 @@ func (m *RequestUnion_HeartbeatTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size())) - n173, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) + n176, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n173 + i += n176 } return i, nil } @@ -13635,11 +13836,11 @@ func (m *RequestUnion_Gc) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size())) - n174, err := m.Gc.MarshalTo(dAtA[i:]) + n177, err := m.Gc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n174 + i += n177 } return i, nil } @@ -13649,11 +13850,11 @@ func (m *RequestUnion_PushTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size())) - n175, err := m.PushTxn.MarshalTo(dAtA[i:]) + n178, err := m.PushTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n175 + i += n178 } return i, nil } @@ -13665,11 +13866,11 @@ func (m *RequestUnion_ResolveIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size())) - n176, err := m.ResolveIntent.MarshalTo(dAtA[i:]) + n179, err := m.ResolveIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n176 + i += n179 } return i, nil } @@ -13681,11 +13882,11 @@ func (m *RequestUnion_ResolveIntentRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size())) - n177, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) + n180, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n177 + i += n180 } return i, nil } @@ -13697,11 +13898,11 @@ func (m *RequestUnion_Merge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size())) - n178, err := m.Merge.MarshalTo(dAtA[i:]) + n181, err := m.Merge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n178 + i += n181 } return i, nil } @@ -13713,11 +13914,11 @@ func (m *RequestUnion_TruncateLog) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size())) - n179, err := m.TruncateLog.MarshalTo(dAtA[i:]) + n182, err := m.TruncateLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n179 + i += n182 } return i, nil } @@ -13729,11 +13930,11 @@ func (m *RequestUnion_RequestLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size())) - n180, err := m.RequestLease.MarshalTo(dAtA[i:]) + n183, err := m.RequestLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n180 + i += n183 } return i, nil } @@ -13745,11 +13946,11 @@ func (m *RequestUnion_ReverseScan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size())) - n181, err := m.ReverseScan.MarshalTo(dAtA[i:]) + n184, err := m.ReverseScan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n181 + i += n184 } return i, nil } @@ -13761,11 +13962,11 @@ func (m *RequestUnion_ComputeChecksum) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size())) - n182, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) + n185, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n182 + i += n185 } return i, nil } @@ -13777,11 +13978,11 @@ func (m *RequestUnion_CheckConsistency) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size())) - n183, err := m.CheckConsistency.MarshalTo(dAtA[i:]) + n186, err := m.CheckConsistency.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n183 + i += n186 } return i, nil } @@ -13793,11 +13994,11 @@ func (m *RequestUnion_InitPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size())) - n184, err := m.InitPut.MarshalTo(dAtA[i:]) + n187, err := m.InitPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n184 + i += n187 } return i, nil } @@ -13809,11 +14010,11 @@ func (m *RequestUnion_TransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TransferLease.Size())) - n185, err := m.TransferLease.MarshalTo(dAtA[i:]) + n188, err := m.TransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n185 + i += n188 } return i, nil } @@ -13825,11 +14026,11 @@ func (m *RequestUnion_AdminTransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size())) - n186, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) + n189, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n186 + i += n189 } return i, nil } @@ -13841,11 +14042,11 @@ func (m *RequestUnion_LeaseInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size())) - n187, err := m.LeaseInfo.MarshalTo(dAtA[i:]) + n190, err := m.LeaseInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n187 + i += n190 } return i, nil } @@ -13857,11 +14058,11 @@ func (m *RequestUnion_WriteBatch) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size())) - n188, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n191, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n188 + i += n191 } return i, nil } @@ -13873,11 +14074,11 @@ func (m *RequestUnion_Export) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Export.Size())) - n189, err := m.Export.MarshalTo(dAtA[i:]) + n192, err := m.Export.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n189 + i += n192 } return i, nil } @@ -13889,11 +14090,11 @@ func (m *RequestUnion_QueryTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size())) - n190, err := m.QueryTxn.MarshalTo(dAtA[i:]) + n193, err := m.QueryTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n190 + i += n193 } return i, nil } @@ -13905,11 +14106,11 @@ func (m *RequestUnion_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Import.Size())) - n191, err := m.Import.MarshalTo(dAtA[i:]) + n194, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n191 + i += n194 } return i, nil } @@ -13921,11 +14122,11 @@ func (m *RequestUnion_AdminChangeReplicas) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size())) - n192, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) + n195, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n192 + i += n195 } return i, nil } @@ -13937,11 +14138,11 @@ func (m *RequestUnion_AdminScatter) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size())) - n193, err := m.AdminScatter.MarshalTo(dAtA[i:]) + n196, err := m.AdminScatter.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n193 + i += n196 } return i, nil } @@ -13953,11 +14154,11 @@ func (m *RequestUnion_AddSstable) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size())) - n194, err := m.AddSstable.MarshalTo(dAtA[i:]) + n197, err := m.AddSstable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n194 + i += n197 } return i, nil } @@ -13969,11 +14170,11 @@ func (m *RequestUnion_ClearRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClearRange.Size())) - n195, err := m.ClearRange.MarshalTo(dAtA[i:]) + n198, err := m.ClearRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n195 + i += n198 } return i, nil } @@ -13985,11 +14186,11 @@ func (m *RequestUnion_RecomputeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecomputeStats.Size())) - n196, err := m.RecomputeStats.MarshalTo(dAtA[i:]) + n199, err := m.RecomputeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n196 + i += n199 } return i, nil } @@ -14001,11 +14202,11 @@ func (m *RequestUnion_Refresh) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Refresh.Size())) - n197, err := m.Refresh.MarshalTo(dAtA[i:]) + n200, err := m.Refresh.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n197 + i += n200 } return i, nil } @@ -14017,11 +14218,11 @@ func (m *RequestUnion_RefreshRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RefreshRange.Size())) - n198, err := m.RefreshRange.MarshalTo(dAtA[i:]) + n201, err := m.RefreshRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n198 + i += n201 } return i, nil } @@ -14033,11 +14234,11 @@ func (m *RequestUnion_QueryIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryIntent.Size())) - n199, err := m.QueryIntent.MarshalTo(dAtA[i:]) + n202, err := m.QueryIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n199 + i += n202 } return i, nil } @@ -14049,11 +14250,11 @@ func (m *RequestUnion_Subsume) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Subsume.Size())) - n200, err := m.Subsume.MarshalTo(dAtA[i:]) + n203, err := m.Subsume.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n200 + i += n203 } return i, nil } @@ -14065,11 +14266,11 @@ func (m *RequestUnion_RangeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeStats.Size())) - n201, err := m.RangeStats.MarshalTo(dAtA[i:]) + n204, err := m.RangeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n201 + i += n204 } return i, nil } @@ -14081,11 +14282,11 @@ func (m *RequestUnion_AdminRelocateRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminRelocateRange.Size())) - n202, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) + n205, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n202 + i += n205 } return i, nil } @@ -14097,11 +14298,11 @@ func (m *RequestUnion_RecoverTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecoverTxn.Size())) - n203, err := m.RecoverTxn.MarshalTo(dAtA[i:]) + n206, err := m.RecoverTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n203 + i += n206 } return i, nil } @@ -14113,11 +14314,11 @@ func (m *RequestUnion_AdminUnsplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminUnsplit.Size())) - n204, err := m.AdminUnsplit.MarshalTo(dAtA[i:]) + n207, err := m.AdminUnsplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n204 + i += n207 } return i, nil } @@ -14129,11 +14330,11 @@ func (m *RequestUnion_RevertRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3 i++ i = encodeVarintApi(dAtA, i, uint64(m.RevertRange.Size())) - n205, err := m.RevertRange.MarshalTo(dAtA[i:]) + n208, err := m.RevertRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n205 + i += n208 } return i, nil } @@ -14145,11 +14346,27 @@ func (m *RequestUnion_AdminVerifyProtectedTimestamp) MarshalTo(dAtA []byte) (int dAtA[i] = 0x3 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminVerifyProtectedTimestamp.Size())) - n206, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:]) + n209, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n206 + i += n209 + } + return i, nil +} +func (m *RequestUnion_Migrate) MarshalTo(dAtA []byte) (int, error) { + i := 0 + if m.Migrate != nil { + dAtA[i] = 0x92 + i++ + dAtA[i] = 0x3 + i++ + i = encodeVarintApi(dAtA, i, uint64(m.Migrate.Size())) + n210, err := m.Migrate.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n210 } return i, nil } @@ -14169,11 +14386,11 @@ func (m *ResponseUnion) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if m.Value != nil { - nn207, err := m.Value.MarshalTo(dAtA[i:]) + nn211, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += nn207 + i += nn211 } return i, nil } @@ -14184,11 +14401,11 @@ func (m *ResponseUnion_Get) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Get.Size())) - n208, err := m.Get.MarshalTo(dAtA[i:]) + n212, err := m.Get.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n208 + i += n212 } return i, nil } @@ -14198,11 +14415,11 @@ func (m *ResponseUnion_Put) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Put.Size())) - n209, err := m.Put.MarshalTo(dAtA[i:]) + n213, err := m.Put.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n209 + i += n213 } return i, nil } @@ -14212,11 +14429,11 @@ func (m *ResponseUnion_ConditionalPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.ConditionalPut.Size())) - n210, err := m.ConditionalPut.MarshalTo(dAtA[i:]) + n214, err := m.ConditionalPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n210 + i += n214 } return i, nil } @@ -14226,11 +14443,11 @@ func (m *ResponseUnion_Increment) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Increment.Size())) - n211, err := m.Increment.MarshalTo(dAtA[i:]) + n215, err := m.Increment.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n211 + i += n215 } return i, nil } @@ -14240,11 +14457,11 @@ func (m *ResponseUnion_Delete) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Delete.Size())) - n212, err := m.Delete.MarshalTo(dAtA[i:]) + n216, err := m.Delete.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n212 + i += n216 } return i, nil } @@ -14254,11 +14471,11 @@ func (m *ResponseUnion_DeleteRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x32 i++ i = encodeVarintApi(dAtA, i, uint64(m.DeleteRange.Size())) - n213, err := m.DeleteRange.MarshalTo(dAtA[i:]) + n217, err := m.DeleteRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n213 + i += n217 } return i, nil } @@ -14268,11 +14485,11 @@ func (m *ResponseUnion_Scan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3a i++ i = encodeVarintApi(dAtA, i, uint64(m.Scan.Size())) - n214, err := m.Scan.MarshalTo(dAtA[i:]) + n218, err := m.Scan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n214 + i += n218 } return i, nil } @@ -14282,11 +14499,11 @@ func (m *ResponseUnion_EndTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x4a i++ i = encodeVarintApi(dAtA, i, uint64(m.EndTxn.Size())) - n215, err := m.EndTxn.MarshalTo(dAtA[i:]) + n219, err := m.EndTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n215 + i += n219 } return i, nil } @@ -14296,11 +14513,11 @@ func (m *ResponseUnion_AdminSplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x52 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminSplit.Size())) - n216, err := m.AdminSplit.MarshalTo(dAtA[i:]) + n220, err := m.AdminSplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n216 + i += n220 } return i, nil } @@ -14310,11 +14527,11 @@ func (m *ResponseUnion_AdminMerge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x5a i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminMerge.Size())) - n217, err := m.AdminMerge.MarshalTo(dAtA[i:]) + n221, err := m.AdminMerge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n217 + i += n221 } return i, nil } @@ -14324,11 +14541,11 @@ func (m *ResponseUnion_HeartbeatTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x62 i++ i = encodeVarintApi(dAtA, i, uint64(m.HeartbeatTxn.Size())) - n218, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) + n222, err := m.HeartbeatTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n218 + i += n222 } return i, nil } @@ -14338,11 +14555,11 @@ func (m *ResponseUnion_Gc) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x6a i++ i = encodeVarintApi(dAtA, i, uint64(m.Gc.Size())) - n219, err := m.Gc.MarshalTo(dAtA[i:]) + n223, err := m.Gc.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n219 + i += n223 } return i, nil } @@ -14352,11 +14569,11 @@ func (m *ResponseUnion_PushTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x72 i++ i = encodeVarintApi(dAtA, i, uint64(m.PushTxn.Size())) - n220, err := m.PushTxn.MarshalTo(dAtA[i:]) + n224, err := m.PushTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n220 + i += n224 } return i, nil } @@ -14368,11 +14585,11 @@ func (m *ResponseUnion_ResolveIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntent.Size())) - n221, err := m.ResolveIntent.MarshalTo(dAtA[i:]) + n225, err := m.ResolveIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n221 + i += n225 } return i, nil } @@ -14384,11 +14601,11 @@ func (m *ResponseUnion_ResolveIntentRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolveIntentRange.Size())) - n222, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) + n226, err := m.ResolveIntentRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n222 + i += n226 } return i, nil } @@ -14400,11 +14617,11 @@ func (m *ResponseUnion_Merge) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.Merge.Size())) - n223, err := m.Merge.MarshalTo(dAtA[i:]) + n227, err := m.Merge.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n223 + i += n227 } return i, nil } @@ -14416,11 +14633,11 @@ func (m *ResponseUnion_TruncateLog) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.TruncateLog.Size())) - n224, err := m.TruncateLog.MarshalTo(dAtA[i:]) + n228, err := m.TruncateLog.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n224 + i += n228 } return i, nil } @@ -14432,11 +14649,11 @@ func (m *ResponseUnion_RequestLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.RequestLease.Size())) - n225, err := m.RequestLease.MarshalTo(dAtA[i:]) + n229, err := m.RequestLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n225 + i += n229 } return i, nil } @@ -14448,11 +14665,11 @@ func (m *ResponseUnion_ReverseScan) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ReverseScan.Size())) - n226, err := m.ReverseScan.MarshalTo(dAtA[i:]) + n230, err := m.ReverseScan.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n226 + i += n230 } return i, nil } @@ -14464,11 +14681,11 @@ func (m *ResponseUnion_ComputeChecksum) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ComputeChecksum.Size())) - n227, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) + n231, err := m.ComputeChecksum.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n227 + i += n231 } return i, nil } @@ -14480,11 +14697,11 @@ func (m *ResponseUnion_CheckConsistency) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.CheckConsistency.Size())) - n228, err := m.CheckConsistency.MarshalTo(dAtA[i:]) + n232, err := m.CheckConsistency.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n228 + i += n232 } return i, nil } @@ -14496,11 +14713,11 @@ func (m *ResponseUnion_InitPut) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.InitPut.Size())) - n229, err := m.InitPut.MarshalTo(dAtA[i:]) + n233, err := m.InitPut.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n229 + i += n233 } return i, nil } @@ -14512,11 +14729,11 @@ func (m *ResponseUnion_AdminTransferLease) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminTransferLease.Size())) - n230, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) + n234, err := m.AdminTransferLease.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n230 + i += n234 } return i, nil } @@ -14528,11 +14745,11 @@ func (m *ResponseUnion_LeaseInfo) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.LeaseInfo.Size())) - n231, err := m.LeaseInfo.MarshalTo(dAtA[i:]) + n235, err := m.LeaseInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n231 + i += n235 } return i, nil } @@ -14544,11 +14761,11 @@ func (m *ResponseUnion_WriteBatch) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.WriteBatch.Size())) - n232, err := m.WriteBatch.MarshalTo(dAtA[i:]) + n236, err := m.WriteBatch.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n232 + i += n236 } return i, nil } @@ -14560,11 +14777,11 @@ func (m *ResponseUnion_Export) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Export.Size())) - n233, err := m.Export.MarshalTo(dAtA[i:]) + n237, err := m.Export.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n233 + i += n237 } return i, nil } @@ -14576,11 +14793,11 @@ func (m *ResponseUnion_QueryTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryTxn.Size())) - n234, err := m.QueryTxn.MarshalTo(dAtA[i:]) + n238, err := m.QueryTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n234 + i += n238 } return i, nil } @@ -14592,11 +14809,11 @@ func (m *ResponseUnion_Import) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Import.Size())) - n235, err := m.Import.MarshalTo(dAtA[i:]) + n239, err := m.Import.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n235 + i += n239 } return i, nil } @@ -14608,11 +14825,11 @@ func (m *ResponseUnion_AdminChangeReplicas) MarshalTo(dAtA []byte) (int, error) dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminChangeReplicas.Size())) - n236, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) + n240, err := m.AdminChangeReplicas.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n236 + i += n240 } return i, nil } @@ -14624,11 +14841,11 @@ func (m *ResponseUnion_AdminScatter) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminScatter.Size())) - n237, err := m.AdminScatter.MarshalTo(dAtA[i:]) + n241, err := m.AdminScatter.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n237 + i += n241 } return i, nil } @@ -14640,11 +14857,11 @@ func (m *ResponseUnion_AddSstable) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AddSstable.Size())) - n238, err := m.AddSstable.MarshalTo(dAtA[i:]) + n242, err := m.AddSstable.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n238 + i += n242 } return i, nil } @@ -14656,11 +14873,11 @@ func (m *ResponseUnion_ClearRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClearRange.Size())) - n239, err := m.ClearRange.MarshalTo(dAtA[i:]) + n243, err := m.ClearRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n239 + i += n243 } return i, nil } @@ -14672,11 +14889,11 @@ func (m *ResponseUnion_RecomputeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecomputeStats.Size())) - n240, err := m.RecomputeStats.MarshalTo(dAtA[i:]) + n244, err := m.RecomputeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n240 + i += n244 } return i, nil } @@ -14688,11 +14905,11 @@ func (m *ResponseUnion_Refresh) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Refresh.Size())) - n241, err := m.Refresh.MarshalTo(dAtA[i:]) + n245, err := m.Refresh.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n241 + i += n245 } return i, nil } @@ -14704,11 +14921,11 @@ func (m *ResponseUnion_RefreshRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RefreshRange.Size())) - n242, err := m.RefreshRange.MarshalTo(dAtA[i:]) + n246, err := m.RefreshRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n242 + i += n246 } return i, nil } @@ -14720,11 +14937,11 @@ func (m *ResponseUnion_QueryIntent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.QueryIntent.Size())) - n243, err := m.QueryIntent.MarshalTo(dAtA[i:]) + n247, err := m.QueryIntent.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n243 + i += n247 } return i, nil } @@ -14736,11 +14953,11 @@ func (m *ResponseUnion_Subsume) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.Subsume.Size())) - n244, err := m.Subsume.MarshalTo(dAtA[i:]) + n248, err := m.Subsume.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n244 + i += n248 } return i, nil } @@ -14752,11 +14969,11 @@ func (m *ResponseUnion_RangeStats) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RangeStats.Size())) - n245, err := m.RangeStats.MarshalTo(dAtA[i:]) + n249, err := m.RangeStats.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n245 + i += n249 } return i, nil } @@ -14768,11 +14985,11 @@ func (m *ResponseUnion_AdminRelocateRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminRelocateRange.Size())) - n246, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) + n250, err := m.AdminRelocateRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n246 + i += n250 } return i, nil } @@ -14784,11 +15001,11 @@ func (m *ResponseUnion_RecoverTxn) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.RecoverTxn.Size())) - n247, err := m.RecoverTxn.MarshalTo(dAtA[i:]) + n251, err := m.RecoverTxn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n247 + i += n251 } return i, nil } @@ -14800,11 +15017,11 @@ func (m *ResponseUnion_AdminUnsplit) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminUnsplit.Size())) - n248, err := m.AdminUnsplit.MarshalTo(dAtA[i:]) + n252, err := m.AdminUnsplit.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n248 + i += n252 } return i, nil } @@ -14816,11 +15033,11 @@ func (m *ResponseUnion_RevertRange) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x3 i++ i = encodeVarintApi(dAtA, i, uint64(m.RevertRange.Size())) - n249, err := m.RevertRange.MarshalTo(dAtA[i:]) + n253, err := m.RevertRange.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n249 + i += n253 } return i, nil } @@ -14832,11 +15049,27 @@ func (m *ResponseUnion_AdminVerifyProtectedTimestamp) MarshalTo(dAtA []byte) (in dAtA[i] = 0x3 i++ i = encodeVarintApi(dAtA, i, uint64(m.AdminVerifyProtectedTimestamp.Size())) - n250, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:]) + n254, err := m.AdminVerifyProtectedTimestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n250 + i += n254 + } + return i, nil +} +func (m *ResponseUnion_Migrate) MarshalTo(dAtA []byte) (int, error) { + i := 0 + if m.Migrate != nil { + dAtA[i] = 0x92 + i++ + dAtA[i] = 0x3 + i++ + i = encodeVarintApi(dAtA, i, uint64(m.Migrate.Size())) + n255, err := m.Migrate.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n255 } return i, nil } @@ -14858,19 +15091,19 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n251, err := m.Timestamp.MarshalTo(dAtA[i:]) + n256, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n251 + i += n256 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Replica.Size())) - n252, err := m.Replica.MarshalTo(dAtA[i:]) + n257, err := m.Replica.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n252 + i += n257 if m.RangeID != 0 { dAtA[i] = 0x18 i++ @@ -14886,11 +15119,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n253, err := m.Txn.MarshalTo(dAtA[i:]) + n258, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n253 + i += n258 } if m.ReadConsistency != 0 { dAtA[i] = 0x30 @@ -14960,11 +15193,11 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1 i++ i = encodeVarintApi(dAtA, i, uint64(m.ClientRangeInfo.Size())) - n254, err := m.ClientRangeInfo.MarshalTo(dAtA[i:]) + n259, err := m.ClientRangeInfo.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n254 + i += n259 } if m.WaitPolicy != 0 { dAtA[i] = 0x90 @@ -15022,11 +15255,11 @@ func (m *BatchRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Header.Size())) - n255, err := m.Header.MarshalTo(dAtA[i:]) + n260, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n255 + i += n260 if len(m.Requests) > 0 { for _, msg := range m.Requests { dAtA[i] = 0x12 @@ -15060,11 +15293,11 @@ func (m *BatchResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.BatchResponse_Header.Size())) - n256, err := m.BatchResponse_Header.MarshalTo(dAtA[i:]) + n261, err := m.BatchResponse_Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n256 + i += n261 if len(m.Responses) > 0 { for _, msg := range m.Responses { dAtA[i] = 0x12 @@ -15099,38 +15332,38 @@ func (m *BatchResponse_Header) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n257, err := m.Error.MarshalTo(dAtA[i:]) + n262, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n257 + i += n262 } dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Timestamp.Size())) - n258, err := m.Timestamp.MarshalTo(dAtA[i:]) + n263, err := m.Timestamp.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n258 + i += n263 if m.Txn != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n259, err := m.Txn.MarshalTo(dAtA[i:]) + n264, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n259 + i += n264 } dAtA[i] = 0x2a i++ i = encodeVarintApi(dAtA, i, uint64(m.Now.Size())) - n260, err := m.Now.MarshalTo(dAtA[i:]) + n265, err := m.Now.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n260 + i += n265 if len(m.CollectedSpans) > 0 { for _, msg := range m.CollectedSpans { dAtA[i] = 0x32 @@ -15245,11 +15478,11 @@ func (m *RangeLookupResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n261, err := m.Error.MarshalTo(dAtA[i:]) + n266, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n261 + i += n266 } return i, nil } @@ -15272,19 +15505,19 @@ func (m *RangeFeedRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Header.Size())) - n262, err := m.Header.MarshalTo(dAtA[i:]) + n267, err := m.Header.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n262 + i += n267 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n263, err := m.Span.MarshalTo(dAtA[i:]) + n268, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n263 + i += n268 if m.WithDiff { dAtA[i] = 0x18 i++ @@ -15322,19 +15555,19 @@ func (m *RangeFeedValue) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Value.Size())) - n264, err := m.Value.MarshalTo(dAtA[i:]) + n269, err := m.Value.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n264 + i += n269 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.PrevValue.Size())) - n265, err := m.PrevValue.MarshalTo(dAtA[i:]) + n270, err := m.PrevValue.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n265 + i += n270 return i, nil } @@ -15356,19 +15589,19 @@ func (m *RangeFeedCheckpoint) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Span.Size())) - n266, err := m.Span.MarshalTo(dAtA[i:]) + n271, err := m.Span.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n266 + i += n271 dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.ResolvedTS.Size())) - n267, err := m.ResolvedTS.MarshalTo(dAtA[i:]) + n272, err := m.ResolvedTS.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n267 + i += n272 return i, nil } @@ -15390,11 +15623,11 @@ func (m *RangeFeedError) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n268, err := m.Error.MarshalTo(dAtA[i:]) + n273, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n268 + i += n273 return i, nil } @@ -15417,31 +15650,31 @@ func (m *RangeFeedEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.Val.Size())) - n269, err := m.Val.MarshalTo(dAtA[i:]) + n274, err := m.Val.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n269 + i += n274 } if m.Checkpoint != nil { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Checkpoint.Size())) - n270, err := m.Checkpoint.MarshalTo(dAtA[i:]) + n275, err := m.Checkpoint.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n270 + i += n275 } if m.Error != nil { dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n271, err := m.Error.MarshalTo(dAtA[i:]) + n276, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n271 + i += n276 } return i, nil } @@ -15544,11 +15777,11 @@ func (m *GossipSubscriptionEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Content.Size())) - n272, err := m.Content.MarshalTo(dAtA[i:]) + n277, err := m.Content.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n272 + i += n277 if len(m.PatternMatched) > 0 { dAtA[i] = 0x1a i++ @@ -15559,11 +15792,11 @@ func (m *GossipSubscriptionEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.Error.Size())) - n273, err := m.Error.MarshalTo(dAtA[i:]) + n278, err := m.Error.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n273 + i += n278 } return i, nil } @@ -15587,11 +15820,11 @@ func (m *JoinNodeRequest) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintApi(dAtA, i, uint64(m.BinaryVersion.Size())) - n274, err := m.BinaryVersion.MarshalTo(dAtA[i:]) + n279, err := m.BinaryVersion.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n274 + i += n279 } return i, nil } @@ -15631,11 +15864,11 @@ func (m *JoinNodeResponse) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintApi(dAtA, i, uint64(m.ActiveVersion.Size())) - n275, err := m.ActiveVersion.MarshalTo(dAtA[i:]) + n280, err := m.ActiveVersion.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n275 + i += n280 } return i, nil } @@ -15664,19 +15897,19 @@ func (m *ContentionEvent) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintApi(dAtA, i, uint64(m.Txn.Size())) - n276, err := m.Txn.MarshalTo(dAtA[i:]) + n281, err := m.Txn.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n276 + i += n281 dAtA[i] = 0x1a i++ i = encodeVarintApi(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration))) - n277, err := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i:]) + n282, err := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i:]) if err != nil { return 0, err } - i += n277 + i += n282 return i, nil } @@ -17386,6 +17619,30 @@ func (m *AdminScatterResponse_Range) 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 *AdminVerifyProtectedTimestampRequest) Size() (n int) { if m == nil { return 0 @@ -18111,6 +18368,18 @@ func (m *RequestUnion_AdminVerifyProtectedTimestamp) Size() (n int) { } return n } +func (m *RequestUnion_Migrate) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Migrate != nil { + l = m.Migrate.Size() + n += 2 + l + sovApi(uint64(l)) + } + return n +} func (m *ResponseUnion) Size() (n int) { if m == nil { return 0 @@ -18639,6 +18908,18 @@ func (m *ResponseUnion_AdminVerifyProtectedTimestamp) Size() (n int) { } return n } +func (m *ResponseUnion_Migrate) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Migrate != nil { + l = m.Migrate.Size() + n += 2 + l + sovApi(uint64(l)) + } + return n +} func (m *Header) Size() (n int) { if m == nil { return 0 @@ -31470,6 +31751,196 @@ func (m *AdminScatterResponse_Range) 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 *AdminVerifyProtectedTimestampRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -34385,6 +34856,38 @@ func (m *RequestUnion) Unmarshal(dAtA []byte) error { } m.Value = &RequestUnion_AdminVerifyProtectedTimestamp{v} iNdEx = postIndex + case 50: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Migrate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &MigrateRequest{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &RequestUnion_Migrate{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -35811,6 +36314,38 @@ func (m *ResponseUnion) Unmarshal(dAtA []byte) error { } m.Value = &ResponseUnion_AdminVerifyProtectedTimestamp{v} iNdEx = postIndex + case 50: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Migrate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &MigrateResponse{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Value = &ResponseUnion_Migrate{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -38505,516 +39040,520 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_290a2f113c3e6c17) } - -var fileDescriptor_api_290a2f113c3e6c17 = []byte{ - // 8124 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0x59, - 0x76, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0x50, 0xfc, 0xd1, 0x95, 0xba, 0x9b, 0xad, 0x99, 0x69, 0xa9, - 0xab, 0xff, 0x7b, 0x67, 0xa4, 0xe9, 0xee, 0x1d, 0xcf, 0x78, 0x7a, 0x3c, 0x6b, 0x91, 0x62, 0x37, - 0x29, 0xb5, 0xd4, 0xea, 0x22, 0xd5, 0xed, 0x19, 0xaf, 0x53, 0x5b, 0xaa, 0xba, 0xa2, 0x6a, 0x45, - 0x56, 0xb1, 0xab, 0x8a, 0xfa, 0x19, 0x20, 0x40, 0x1c, 0x1b, 0x89, 0x81, 0x00, 0x0b, 0x3f, 0x38, - 0xc8, 0x1a, 0x4e, 0xe2, 0x75, 0x1c, 0xc7, 0x01, 0x82, 0x20, 0x01, 0x12, 0xc4, 0x41, 0x90, 0xd8, - 0x2f, 0x46, 0xb2, 0x08, 0x0c, 0x64, 0xfd, 0x14, 0x23, 0x40, 0x14, 0x5b, 0x9b, 0xb7, 0x04, 0x46, - 0x90, 0x97, 0x00, 0xf3, 0x10, 0x04, 0xf7, 0xa7, 0xfe, 0xc8, 0x22, 0x45, 0xf5, 0xd6, 0x20, 0x03, - 0xec, 0x8b, 0xc4, 0x3a, 0x75, 0xcf, 0xa9, 0x7b, 0xcf, 0xbd, 0xf7, 0xdc, 0xf3, 0xdd, 0x3a, 0xf7, - 0x14, 0xcc, 0x5a, 0xa6, 0xa2, 0xee, 0x77, 0x77, 0x57, 0x94, 0xae, 0xbe, 0xdc, 0xb5, 0x4c, 0xc7, - 0x44, 0xb3, 0xaa, 0xa9, 0x1e, 0x50, 0xf2, 0x32, 0xbf, 0xb9, 0x70, 0xff, 0xe0, 0x70, 0xe5, 0xe0, - 0xd0, 0xc6, 0xd6, 0x21, 0xb6, 0x56, 0x54, 0xd3, 0x50, 0x7b, 0x96, 0x85, 0x0d, 0xf5, 0x64, 0xa5, - 0x6d, 0xaa, 0x07, 0xf4, 0x8f, 0x6e, 0xb4, 0x18, 0xfb, 0x02, 0x72, 0x25, 0x6a, 0x8a, 0xa3, 0x70, - 0xda, 0xbc, 0x4b, 0xc3, 0x96, 0x65, 0x5a, 0x36, 0xa7, 0x5e, 0x76, 0xa9, 0x1d, 0xec, 0x28, 0x81, - 0xd2, 0x6f, 0xd9, 0x8e, 0x69, 0x29, 0x2d, 0xbc, 0x82, 0x8d, 0x96, 0x6e, 0x60, 0x52, 0xe0, 0x50, - 0x55, 0xf9, 0xcd, 0xb7, 0x23, 0x6f, 0x3e, 0xe2, 0x77, 0x4b, 0x3d, 0x47, 0x6f, 0xaf, 0xec, 0xb7, - 0xd5, 0x15, 0x47, 0xef, 0x60, 0xdb, 0x51, 0x3a, 0x5d, 0x7e, 0xe7, 0x3e, 0xbd, 0xe3, 0x58, 0x8a, - 0xaa, 0x1b, 0x2d, 0xf7, 0x7f, 0x77, 0x77, 0xc5, 0xc2, 0xaa, 0x69, 0x69, 0x58, 0x93, 0xed, 0xae, - 0x62, 0xb8, 0xd5, 0x6d, 0x99, 0x2d, 0x93, 0xfe, 0x5c, 0x21, 0xbf, 0x38, 0xf5, 0x5a, 0xcb, 0x34, - 0x5b, 0x6d, 0xbc, 0x42, 0xaf, 0x76, 0x7b, 0x7b, 0x2b, 0x5a, 0xcf, 0x52, 0x1c, 0xdd, 0xe4, 0x5c, - 0xe2, 0xbf, 0x12, 0x20, 0x27, 0xe1, 0xd7, 0x3d, 0x6c, 0x3b, 0x35, 0xac, 0x68, 0xd8, 0x42, 0x57, - 0x21, 0x79, 0x80, 0x4f, 0x4a, 0xc9, 0x25, 0xe1, 0xee, 0x4c, 0x79, 0xfa, 0xcb, 0xd3, 0xc5, 0xe4, - 0x06, 0x3e, 0x91, 0x08, 0x0d, 0x2d, 0xc1, 0x34, 0x36, 0x34, 0x99, 0xdc, 0x4e, 0x85, 0x6f, 0x4f, - 0x61, 0x43, 0xdb, 0xc0, 0x27, 0xe8, 0xdb, 0x90, 0xb6, 0x89, 0x34, 0x43, 0xc5, 0xa5, 0xc9, 0x25, - 0xe1, 0xee, 0x64, 0xf9, 0xe7, 0xbf, 0x3c, 0x5d, 0xfc, 0xa4, 0xa5, 0x3b, 0xfb, 0xbd, 0xdd, 0x65, - 0xd5, 0xec, 0xac, 0x78, 0xfd, 0xa4, 0xed, 0xfa, 0xbf, 0x57, 0xba, 0x07, 0xad, 0x95, 0x7e, 0x1d, - 0x2d, 0x37, 0x8f, 0x8d, 0x06, 0x7e, 0x2d, 0x79, 0x12, 0xd7, 0x53, 0x69, 0xa1, 0x98, 0x58, 0x4f, - 0xa5, 0x13, 0xc5, 0xa4, 0xf8, 0xbb, 0x49, 0xc8, 0x4b, 0xd8, 0xee, 0x9a, 0x86, 0x8d, 0x79, 0xcd, - 0xdf, 0x87, 0xa4, 0x73, 0x6c, 0xd0, 0x9a, 0x67, 0x1f, 0x5e, 0x5b, 0x1e, 0x18, 0x11, 0xcb, 0x4d, - 0x4b, 0x31, 0x6c, 0x45, 0x25, 0xcd, 0x97, 0x48, 0x51, 0xf4, 0x11, 0x64, 0x2d, 0x6c, 0xf7, 0x3a, - 0x98, 0x2a, 0x92, 0x36, 0x2a, 0xfb, 0xf0, 0x4a, 0x04, 0x67, 0xa3, 0xab, 0x18, 0x12, 0xb0, 0xb2, - 0xe4, 0x37, 0xba, 0x0a, 0x69, 0xa3, 0xd7, 0x21, 0xaa, 0xb0, 0x69, 0x43, 0x93, 0xd2, 0xb4, 0xd1, - 0xeb, 0x6c, 0xe0, 0x13, 0x1b, 0xfd, 0x02, 0x5c, 0xd6, 0x70, 0xd7, 0xc2, 0xaa, 0xe2, 0x60, 0x4d, - 0xb6, 0x14, 0xa3, 0x85, 0x65, 0xdd, 0xd8, 0x33, 0xed, 0xd2, 0xd4, 0x52, 0xf2, 0x6e, 0xf6, 0xe1, - 0xdb, 0x11, 0xf2, 0x25, 0x52, 0xaa, 0x6e, 0xec, 0x99, 0xe5, 0xd4, 0x0f, 0x4f, 0x17, 0x27, 0xa4, - 0x79, 0x5f, 0x82, 0x77, 0xcb, 0x46, 0x0d, 0xc8, 0xf1, 0xea, 0x5a, 0x58, 0xb1, 0x4d, 0xa3, 0x34, - 0xbd, 0x24, 0xdc, 0xcd, 0x3f, 0x5c, 0x8e, 0x12, 0x18, 0x52, 0x0d, 0xb9, 0xec, 0x75, 0xb0, 0x44, - 0xb9, 0xa4, 0x19, 0x2b, 0x70, 0x85, 0xde, 0x82, 0x0c, 0x69, 0xc9, 0xee, 0x89, 0x83, 0xed, 0x52, - 0x9a, 0x36, 0x85, 0x34, 0xad, 0x4c, 0xae, 0xc5, 0x4f, 0x61, 0x26, 0xc8, 0x8a, 0x10, 0xe4, 0xa5, - 0x6a, 0x63, 0x67, 0xb3, 0x2a, 0xef, 0x6c, 0x6d, 0x6c, 0x3d, 0x7f, 0xb5, 0x55, 0x9c, 0x40, 0xf3, - 0x50, 0xe4, 0xb4, 0x8d, 0xea, 0x67, 0xf2, 0xb3, 0xfa, 0x66, 0xbd, 0x59, 0x14, 0x16, 0x52, 0xbf, - 0xf6, 0xbb, 0xd7, 0x26, 0xc4, 0x6d, 0x80, 0xa7, 0xd8, 0xe1, 0x03, 0x0c, 0x95, 0x61, 0x6a, 0x9f, - 0xd6, 0xa7, 0x24, 0x50, 0x4d, 0x2f, 0x45, 0x56, 0x3c, 0x30, 0x18, 0xcb, 0x69, 0xa2, 0x8d, 0x1f, - 0x9d, 0x2e, 0x0a, 0x12, 0xe7, 0x14, 0xff, 0x48, 0x80, 0x2c, 0x15, 0xc9, 0xda, 0x87, 0x2a, 0x7d, - 0x32, 0xaf, 0x9f, 0xab, 0x8c, 0x41, 0xa1, 0x68, 0x19, 0x26, 0x0f, 0x95, 0x76, 0x0f, 0x97, 0x12, - 0x54, 0x46, 0x29, 0x42, 0xc6, 0x4b, 0x72, 0x5f, 0x62, 0xc5, 0xd0, 0x63, 0x98, 0xd1, 0x0d, 0x07, - 0x1b, 0x8e, 0xcc, 0xd8, 0x92, 0xe7, 0xb0, 0x65, 0x59, 0x69, 0x7a, 0x21, 0xfe, 0x4b, 0x01, 0x60, - 0xbb, 0x17, 0xa7, 0x52, 0xd0, 0x37, 0xc7, 0xac, 0x3f, 0x1f, 0x5d, 0xbc, 0x15, 0x97, 0x61, 0x4a, - 0x37, 0xda, 0xba, 0xc1, 0xea, 0x9f, 0x96, 0xf8, 0x15, 0x9a, 0x87, 0xc9, 0xdd, 0xb6, 0x6e, 0x68, - 0x74, 0x3e, 0xa4, 0x25, 0x76, 0x21, 0x4a, 0x90, 0xa5, 0xb5, 0x8e, 0x51, 0xef, 0xe2, 0xbf, 0x4f, - 0xc0, 0xa5, 0x8a, 0x69, 0x68, 0x3a, 0x99, 0x92, 0x4a, 0xfb, 0x6b, 0xa1, 0x95, 0x75, 0x08, 0x4c, - 0x3e, 0x19, 0x1f, 0x77, 0xc7, 0xec, 0x63, 0xe4, 0x73, 0x55, 0x8f, 0xbb, 0x94, 0x16, 0xad, 0x49, - 0xf4, 0x4d, 0xb8, 0xa2, 0xb4, 0xdb, 0xe6, 0x91, 0xac, 0xef, 0xc9, 0x9a, 0x89, 0x6d, 0xd9, 0x30, - 0x1d, 0x19, 0x1f, 0xeb, 0xb6, 0x43, 0x4d, 0x49, 0x5a, 0x9a, 0xa3, 0xb7, 0xeb, 0x7b, 0x6b, 0x26, - 0xb6, 0xb7, 0x4c, 0xa7, 0x4a, 0x6e, 0x91, 0x79, 0x4a, 0x2a, 0xc3, 0xe6, 0xe9, 0x14, 0x31, 0xbf, - 0x52, 0x1a, 0x1f, 0x77, 0xd9, 0x3c, 0xfd, 0x25, 0xb8, 0xdc, 0xaf, 0xc7, 0x38, 0xfb, 0xe9, 0x4f, - 0x04, 0xc8, 0xd7, 0x0d, 0xdd, 0xf9, 0x5a, 0x74, 0x90, 0xa7, 0xd4, 0x64, 0x50, 0xa9, 0xf7, 0xa1, - 0xb8, 0xa7, 0xe8, 0xed, 0xe7, 0x46, 0xd3, 0xec, 0xec, 0xda, 0x8e, 0x69, 0x60, 0x9b, 0x6b, 0x7d, - 0x80, 0x2e, 0xbe, 0x84, 0x82, 0xd7, 0x9a, 0x38, 0xd5, 0xe4, 0x40, 0xb1, 0x6e, 0xa8, 0x16, 0xee, - 0x60, 0x23, 0x56, 0x3d, 0xbd, 0x0d, 0x19, 0xdd, 0x95, 0x4b, 0x75, 0x95, 0x94, 0x7c, 0x82, 0xd8, - 0x83, 0xd9, 0xc0, 0x53, 0xe3, 0x34, 0x8b, 0x64, 0x69, 0xc0, 0x47, 0xb2, 0xdf, 0x47, 0x64, 0x69, - 0xc0, 0x47, 0xcc, 0x8c, 0x35, 0x20, 0xb7, 0x86, 0xdb, 0xd8, 0xc1, 0x71, 0x5a, 0xf7, 0x1d, 0xc8, - 0xbb, 0x42, 0xe3, 0xec, 0x98, 0xbf, 0x23, 0x00, 0xe2, 0x72, 0xc9, 0x6a, 0x1a, 0x67, 0xdf, 0x2c, - 0x12, 0x17, 0xc2, 0xe9, 0x59, 0x06, 0xf3, 0x05, 0xd8, 0x98, 0x04, 0x46, 0xa2, 0xee, 0x80, 0x6f, - 0x65, 0x53, 0x41, 0x2b, 0xcb, 0xdd, 0x98, 0x23, 0x98, 0x0b, 0x55, 0x2c, 0xde, 0xee, 0x4b, 0xd1, - 0x3a, 0x25, 0x96, 0x92, 0x41, 0x5f, 0x8d, 0x12, 0xc5, 0xef, 0x0b, 0x30, 0x5b, 0x69, 0x63, 0xc5, - 0x8a, 0x5d, 0x23, 0xdf, 0x82, 0xb4, 0x86, 0x15, 0x8d, 0x36, 0x99, 0x4d, 0xec, 0x77, 0x02, 0x52, - 0x88, 0x47, 0xbb, 0xbc, 0xdf, 0x56, 0x97, 0x9b, 0xae, 0xaf, 0xcb, 0x67, 0xb7, 0xc7, 0x24, 0x7e, - 0x06, 0x28, 0x58, 0xb3, 0x38, 0x07, 0xc2, 0xdf, 0x17, 0x00, 0x49, 0xf8, 0x10, 0x5b, 0x4e, 0xec, - 0xcd, 0x5e, 0x83, 0xac, 0xa3, 0x58, 0x2d, 0xec, 0xc8, 0xc4, 0x8b, 0xbf, 0x48, 0xcb, 0x81, 0xf1, - 0x11, 0xb2, 0xf8, 0x39, 0xcc, 0x85, 0xea, 0x17, 0x67, 0xe3, 0xff, 0x97, 0x00, 0xd9, 0x86, 0xaa, - 0x18, 0x71, 0xb6, 0xfa, 0x53, 0xc8, 0xda, 0xaa, 0x62, 0xc8, 0x7b, 0xa6, 0xd5, 0x51, 0x1c, 0x3a, - 0xc4, 0xf3, 0xa1, 0x56, 0x7b, 0x1e, 0xb4, 0xaa, 0x18, 0x4f, 0x68, 0x21, 0x09, 0x6c, 0xef, 0x37, - 0x7a, 0x01, 0xd9, 0x03, 0x7c, 0x22, 0x73, 0x34, 0x46, 0xd7, 0xbf, 0xfc, 0xc3, 0xf7, 0x03, 0xfc, - 0x07, 0x87, 0xcb, 0x2e, 0x88, 0x5b, 0x0e, 0x80, 0xb8, 0x65, 0xc2, 0xb1, 0xdc, 0x70, 0x2c, 0x6c, - 0xb4, 0x9c, 0x7d, 0x09, 0x0e, 0xf0, 0xc9, 0x33, 0x26, 0x83, 0x4d, 0xac, 0xf5, 0x54, 0x3a, 0x59, - 0x4c, 0x89, 0xff, 0x47, 0x80, 0x19, 0xd6, 0xe4, 0x38, 0x27, 0xd6, 0x07, 0x90, 0xb2, 0xcc, 0x23, - 0x36, 0xb1, 0xb2, 0x0f, 0xdf, 0x8a, 0x10, 0xb1, 0x81, 0x4f, 0x82, 0x2b, 0x17, 0x2d, 0x8e, 0xca, - 0xc0, 0xfd, 0x40, 0x99, 0x72, 0x27, 0xc7, 0xe5, 0x06, 0xc6, 0x25, 0x11, 0x19, 0x77, 0xa0, 0xb0, - 0xab, 0x38, 0xea, 0xbe, 0x6c, 0xf1, 0x4a, 0x92, 0x55, 0x2e, 0x79, 0x77, 0x46, 0xca, 0x53, 0xb2, - 0x5b, 0x75, 0x9b, 0xb4, 0x9c, 0x8d, 0x74, 0x1b, 0xff, 0x94, 0xf5, 0xf9, 0xff, 0x15, 0xf8, 0x1c, - 0x72, 0x5b, 0xfe, 0xd3, 0xd6, 0xf5, 0xbf, 0x99, 0x80, 0x2b, 0x95, 0x7d, 0xac, 0x1e, 0x54, 0x4c, - 0xc3, 0xd6, 0x6d, 0x87, 0xe8, 0x2e, 0xce, 0xfe, 0x7f, 0x0b, 0x32, 0x47, 0xba, 0xb3, 0x2f, 0x6b, - 0xfa, 0xde, 0x1e, 0xb5, 0x73, 0x69, 0x29, 0x4d, 0x08, 0x6b, 0xfa, 0xde, 0x1e, 0x7a, 0x04, 0xa9, - 0x8e, 0xa9, 0x31, 0x77, 0x39, 0xff, 0x70, 0x31, 0x42, 0x3c, 0xad, 0x9a, 0xdd, 0xeb, 0x6c, 0x9a, - 0x1a, 0x96, 0x68, 0x61, 0x74, 0x0d, 0x40, 0x25, 0xd4, 0xae, 0xa9, 0x1b, 0x0e, 0x5f, 0x27, 0x03, - 0x14, 0x54, 0x83, 0x8c, 0x83, 0xad, 0x8e, 0x6e, 0x28, 0x0e, 0x2e, 0x4d, 0x52, 0xe5, 0xdd, 0x8c, - 0xac, 0x78, 0xb7, 0xad, 0xab, 0xca, 0x1a, 0xb6, 0x55, 0x4b, 0xef, 0x3a, 0xa6, 0xc5, 0xb5, 0xe8, - 0x33, 0x8b, 0xdf, 0x4b, 0x41, 0x69, 0x50, 0x37, 0x71, 0x8e, 0x90, 0x6d, 0x98, 0x22, 0xf8, 0xba, - 0xed, 0xf0, 0x31, 0xf2, 0x70, 0x98, 0x0a, 0x22, 0x6a, 0x40, 0x71, 0x7a, 0xdb, 0xe1, 0xd5, 0xe6, - 0x72, 0x16, 0xfe, 0xad, 0x00, 0x53, 0xec, 0x06, 0x7a, 0x00, 0x69, 0xbe, 0xa1, 0xa0, 0xd1, 0x3a, - 0x26, 0xcb, 0x97, 0xcf, 0x4e, 0x17, 0xa7, 0xd9, 0x1e, 0xc1, 0xda, 0x97, 0xfe, 0x4f, 0x69, 0x9a, - 0x96, 0xab, 0x6b, 0xa4, 0xb7, 0x6c, 0x47, 0xb1, 0x1c, 0xba, 0x6d, 0x93, 0x60, 0xb8, 0x81, 0x12, - 0x36, 0xf0, 0x09, 0x5a, 0x87, 0x29, 0xdb, 0x51, 0x9c, 0x9e, 0xcd, 0xfb, 0xeb, 0x42, 0x95, 0x6d, - 0x50, 0x4e, 0x89, 0x4b, 0x20, 0x8e, 0x8e, 0x86, 0x1d, 0x45, 0x6f, 0xd3, 0x0e, 0xcc, 0x48, 0xfc, - 0x4a, 0xfc, 0x2d, 0x01, 0xa6, 0x58, 0x51, 0x74, 0x05, 0xe6, 0xa4, 0xd5, 0xad, 0xa7, 0x55, 0xb9, - 0xbe, 0xb5, 0x56, 0x6d, 0x56, 0xa5, 0xcd, 0xfa, 0xd6, 0x6a, 0xb3, 0x5a, 0x9c, 0x40, 0x97, 0x01, - 0xb9, 0x37, 0x2a, 0xcf, 0xb7, 0x1a, 0xf5, 0x46, 0xb3, 0xba, 0xd5, 0x2c, 0x0a, 0x74, 0x6f, 0x81, - 0xd2, 0x03, 0xd4, 0x04, 0xba, 0x09, 0x4b, 0xfd, 0x54, 0xb9, 0xd1, 0x5c, 0x6d, 0x36, 0xe4, 0x6a, - 0xa3, 0x59, 0xdf, 0x5c, 0x6d, 0x56, 0xd7, 0x8a, 0xc9, 0x11, 0xa5, 0xc8, 0x43, 0x24, 0xa9, 0x5a, - 0x69, 0x16, 0x53, 0xa2, 0x03, 0x97, 0x24, 0xac, 0x9a, 0x9d, 0x6e, 0xcf, 0xc1, 0xa4, 0x96, 0x76, - 0x9c, 0x33, 0xe5, 0x0a, 0x4c, 0x6b, 0xd6, 0x89, 0x6c, 0xf5, 0x0c, 0x3e, 0x4f, 0xa6, 0x34, 0xeb, - 0x44, 0xea, 0x19, 0xe2, 0x3f, 0x17, 0xe0, 0x72, 0xff, 0x63, 0xe3, 0x1c, 0x84, 0x2f, 0x20, 0xab, - 0x68, 0x1a, 0xd6, 0x64, 0x0d, 0xb7, 0x1d, 0x85, 0x3b, 0x23, 0xf7, 0x03, 0x92, 0xf8, 0x66, 0xdb, - 0xb2, 0xb7, 0xd9, 0xb6, 0xf9, 0xb2, 0x52, 0xa1, 0x15, 0x59, 0x23, 0x1c, 0xae, 0xf9, 0xa1, 0x42, - 0x28, 0x45, 0xfc, 0x9f, 0x29, 0xc8, 0x55, 0x0d, 0xad, 0x79, 0x1c, 0xeb, 0x5a, 0x72, 0x19, 0xa6, - 0x54, 0xb3, 0xd3, 0xd1, 0x1d, 0x57, 0x41, 0xec, 0x0a, 0xfd, 0x6c, 0xc0, 0x89, 0x4c, 0x8e, 0xe1, - 0x4a, 0xf9, 0xee, 0x23, 0xfa, 0x0e, 0x5c, 0x21, 0x56, 0xd3, 0x32, 0x94, 0xb6, 0xcc, 0xa4, 0xc9, - 0x8e, 0xa5, 0xb7, 0x5a, 0xd8, 0xe2, 0x1b, 0x7c, 0x77, 0x23, 0xea, 0x59, 0xe7, 0x1c, 0x15, 0xca, - 0xd0, 0x64, 0xe5, 0xa5, 0x4b, 0x7a, 0x14, 0x19, 0x7d, 0x02, 0x40, 0x96, 0x22, 0xba, 0x69, 0x68, - 0x73, 0x7b, 0x34, 0x6c, 0xd7, 0xd0, 0x35, 0x41, 0x84, 0x81, 0x5c, 0xdb, 0x68, 0x85, 0x20, 0x86, - 0xd7, 0x3d, 0xdd, 0xc2, 0xf2, 0x83, 0xae, 0x4a, 0xa1, 0x7c, 0xba, 0x9c, 0x3f, 0x3b, 0x5d, 0x04, - 0x89, 0x91, 0x1f, 0x6c, 0x57, 0x08, 0x82, 0x60, 0xbf, 0xbb, 0x2a, 0x7a, 0x05, 0xf7, 0x02, 0x3b, - 0x12, 0x64, 0xe5, 0xe5, 0xcd, 0x52, 0x1c, 0x79, 0x5f, 0x6f, 0xed, 0x63, 0x4b, 0xf6, 0x36, 0x8e, - 0xe9, 0x0e, 0x5e, 0x5a, 0xba, 0xe9, 0x33, 0x54, 0x14, 0x83, 0xd5, 0x7e, 0xd5, 0xa9, 0xd1, 0xc2, - 0x9e, 0xce, 0x88, 0xf2, 0xbb, 0xa6, 0x6e, 0x9b, 0x46, 0x29, 0xc3, 0x94, 0xcf, 0xae, 0xd0, 0x3d, - 0x28, 0x3a, 0xc7, 0x86, 0xbc, 0x8f, 0x15, 0xcb, 0xd9, 0xc5, 0x8a, 0x43, 0x56, 0x69, 0xa0, 0x25, - 0x0a, 0xce, 0xb1, 0x51, 0x0b, 0x90, 0xd1, 0x0b, 0x28, 0xea, 0x86, 0xbc, 0xd7, 0xd6, 0x5b, 0xfb, - 0x8e, 0x7c, 0x64, 0xe9, 0x0e, 0xb6, 0x4b, 0xb3, 0x54, 0x21, 0x51, 0xe3, 0xb6, 0xc1, 0x77, 0x72, - 0xb5, 0x57, 0xa4, 0x24, 0x57, 0x4d, 0x5e, 0x37, 0x9e, 0x50, 0x7e, 0x4a, 0xb4, 0xd7, 0x53, 0xe9, - 0xe9, 0x62, 0x5a, 0xfc, 0xaf, 0x02, 0xe4, 0xdd, 0xe1, 0x16, 0xe7, 0xcc, 0xb8, 0x0b, 0x45, 0xd3, - 0xc0, 0x72, 0x77, 0x5f, 0xb1, 0x31, 0xd7, 0x23, 0x5f, 0x70, 0xf2, 0xa6, 0x81, 0xb7, 0x09, 0x99, - 0xa9, 0x0b, 0x6d, 0xc3, 0xac, 0xed, 0x28, 0x2d, 0xdd, 0x68, 0x05, 0xd4, 0x3b, 0x39, 0xbe, 0x5b, - 0x5f, 0xe4, 0xdc, 0x1e, 0x3d, 0xe4, 0xa5, 0xfc, 0xa9, 0x00, 0xb3, 0xab, 0x5a, 0x47, 0x37, 0x1a, - 0xdd, 0xb6, 0x1e, 0xeb, 0x6e, 0xc1, 0x4d, 0xc8, 0xd8, 0x44, 0xa6, 0x6f, 0xf0, 0x7d, 0xec, 0x97, - 0xa6, 0x77, 0x88, 0xe5, 0x7f, 0x06, 0x05, 0x7c, 0xdc, 0xd5, 0xd9, 0xcb, 0x00, 0x06, 0x59, 0x52, - 0xe3, 0xb7, 0x2d, 0xef, 0xf3, 0x92, 0x5b, 0xbc, 0x4d, 0x9f, 0x01, 0x0a, 0x36, 0x29, 0x4e, 0xec, - 0xf2, 0x19, 0xcc, 0x51, 0xd1, 0x3b, 0x86, 0x1d, 0xb3, 0xbe, 0xc4, 0x5f, 0x84, 0xf9, 0xb0, 0xe8, - 0x38, 0xeb, 0xfd, 0x8a, 0xf7, 0xf2, 0x26, 0xb6, 0x62, 0x85, 0x9b, 0x9e, 0xae, 0xb9, 0xe0, 0x38, - 0xeb, 0xfc, 0xab, 0x02, 0x5c, 0xa5, 0xb2, 0xe9, 0xfb, 0x92, 0x3d, 0x6c, 0x3d, 0xc3, 0x8a, 0x1d, - 0x2b, 0x56, 0xbe, 0x01, 0x53, 0x0c, 0xf3, 0xd2, 0xf1, 0x39, 0x59, 0xce, 0x12, 0xcf, 0xa5, 0xe1, - 0x98, 0x16, 0xf1, 0x5c, 0xf8, 0x2d, 0x51, 0x81, 0x85, 0xa8, 0x5a, 0xc4, 0xbc, 0x1d, 0x30, 0xcb, - 0x9d, 0x46, 0x32, 0x94, 0x2b, 0xfb, 0xc4, 0x67, 0x42, 0x55, 0xc8, 0xaa, 0xf4, 0x97, 0xec, 0x9c, - 0x74, 0x31, 0x95, 0x9f, 0x1f, 0xe5, 0x6f, 0x32, 0xb6, 0xe6, 0x49, 0x17, 0x13, 0xa7, 0xd5, 0xfd, - 0x4d, 0x14, 0x15, 0x68, 0xe4, 0x48, 0x8f, 0x95, 0xce, 0x23, 0x5a, 0xd6, 0x75, 0xfd, 0xb8, 0x0e, - 0xfe, 0x45, 0x92, 0x2b, 0x81, 0x3d, 0x83, 0x17, 0x8f, 0xd5, 0x47, 0xf9, 0x3c, 0xf4, 0xba, 0x2a, - 0xd8, 0xf0, 0xc4, 0x05, 0x1a, 0x1e, 0xd8, 0x33, 0xf7, 0xa9, 0xe8, 0x33, 0x08, 0xec, 0x8a, 0xcb, - 0xac, 0x4d, 0x2e, 0xfa, 0xb9, 0x88, 0x3a, 0x66, 0x7d, 0x29, 0x8c, 0x6e, 0xa3, 0x0a, 0xa4, 0xf1, - 0x71, 0x57, 0xd6, 0xb0, 0xad, 0x72, 0xc3, 0x25, 0x0e, 0x7b, 0xaf, 0x36, 0x80, 0x07, 0xa6, 0xf1, - 0x71, 0x97, 0x10, 0xd1, 0x0e, 0x59, 0xbd, 0x5c, 0x57, 0x81, 0x56, 0xdb, 0x3e, 0x1f, 0x5e, 0xf8, - 0x23, 0x85, 0x8b, 0x2b, 0x78, 0x5e, 0x02, 0x13, 0x21, 0xfe, 0x40, 0x80, 0xb7, 0x22, 0x7b, 0x2d, - 0xce, 0x85, 0xec, 0x13, 0x48, 0xd1, 0xc6, 0x27, 0x2e, 0xd8, 0x78, 0xca, 0x25, 0xfe, 0x9e, 0x3b, - 0xc7, 0x25, 0xdc, 0x36, 0x89, 0x62, 0xbf, 0x82, 0xfd, 0xb0, 0x69, 0xb7, 0xc3, 0x13, 0x17, 0xee, - 0x70, 0x97, 0xd5, 0x33, 0x02, 0x7d, 0xd5, 0x8c, 0xd3, 0x08, 0xfc, 0xba, 0x00, 0x73, 0x9e, 0x4f, - 0x13, 0xb3, 0x7b, 0xfb, 0x01, 0x24, 0x0d, 0xf3, 0xe8, 0x22, 0x9b, 0x81, 0xa4, 0x3c, 0x59, 0x92, - 0xc2, 0x35, 0x8a, 0xb3, 0xbd, 0xff, 0x2e, 0x01, 0x99, 0xa7, 0x95, 0x38, 0x5b, 0xf9, 0x09, 0xdf, - 0x68, 0x66, 0x13, 0x3b, 0x6a, 0x28, 0x7a, 0xcf, 0x5b, 0x7e, 0x5a, 0xd9, 0xc0, 0x27, 0xee, 0x50, - 0x24, 0x5c, 0x68, 0x15, 0x32, 0xce, 0xbe, 0x85, 0xed, 0x7d, 0xb3, 0xad, 0x5d, 0xc4, 0x07, 0xf1, - 0xb9, 0x16, 0x30, 0x4c, 0x52, 0xb9, 0x6e, 0xf0, 0x82, 0x10, 0x11, 0xbc, 0x40, 0x1e, 0xe3, 0xb9, - 0x71, 0x89, 0x8b, 0x3c, 0x26, 0xe0, 0xbf, 0x4d, 0x16, 0xa7, 0xc4, 0x17, 0x00, 0xa4, 0x39, 0x71, - 0x76, 0xc9, 0xdf, 0x48, 0x42, 0x7e, 0xbb, 0x67, 0xef, 0xc7, 0x3c, 0xfa, 0x2a, 0x00, 0xdd, 0x9e, - 0x4d, 0xf1, 0xc1, 0xb1, 0xc1, 0xdb, 0x7c, 0x4e, 0x5c, 0x84, 0xdb, 0x68, 0xc6, 0xd7, 0x3c, 0x36, - 0x50, 0x8d, 0x0b, 0xc1, 0xb2, 0x1f, 0x5c, 0x71, 0x63, 0x14, 0x92, 0x6c, 0x1e, 0x1b, 0x9b, 0xd8, - 0x83, 0x90, 0x4c, 0x12, 0x26, 0x92, 0x3e, 0x81, 0x69, 0x72, 0x21, 0x3b, 0xe6, 0x45, 0xba, 0x79, - 0x8a, 0xf0, 0x34, 0x4d, 0xf4, 0x18, 0x32, 0x8c, 0x9b, 0x2c, 0x4d, 0x53, 0x74, 0x69, 0x8a, 0x6a, - 0x0b, 0x57, 0x23, 0x5d, 0x94, 0xd2, 0x94, 0x95, 0x2c, 0x44, 0xf3, 0x30, 0xb9, 0x67, 0x5a, 0x2a, - 0xa6, 0x11, 0x13, 0x69, 0x89, 0x5d, 0xb0, 0xfe, 0x5c, 0x4f, 0xa5, 0xd3, 0xc5, 0xcc, 0x7a, 0x2a, - 0x9d, 0x29, 0x82, 0xf8, 0x5b, 0x02, 0x14, 0xbc, 0x8e, 0x88, 0xd3, 0x5a, 0x57, 0x42, 0x5a, 0xbc, - 0x78, 0x57, 0x10, 0x05, 0x8a, 0xff, 0x81, 0xba, 0x2b, 0xaa, 0x79, 0x48, 0x7b, 0x26, 0xce, 0x91, - 0xf2, 0x98, 0x85, 0xce, 0x24, 0x2e, 0xda, 0xbb, 0x34, 0x8a, 0xe6, 0x01, 0xcc, 0xeb, 0x1d, 0x62, - 0xc7, 0x75, 0xa7, 0x7d, 0xc2, 0x31, 0x95, 0x83, 0xdd, 0xf7, 0xb3, 0x73, 0xfe, 0xbd, 0x8a, 0x7b, - 0x4b, 0xfc, 0x87, 0x74, 0x77, 0xda, 0x6f, 0x49, 0x9c, 0xaa, 0xae, 0x43, 0xce, 0x62, 0xa2, 0x89, - 0xcf, 0x71, 0x41, 0x6d, 0xcf, 0x78, 0xac, 0x44, 0xe1, 0xbf, 0x93, 0x80, 0xc2, 0x8b, 0x1e, 0xb6, - 0x4e, 0xbe, 0x4e, 0xea, 0xbe, 0x0d, 0x85, 0x23, 0x45, 0x77, 0xe4, 0x3d, 0xd3, 0x92, 0x7b, 0x5d, - 0x4d, 0x71, 0xdc, 0xf8, 0x8d, 0x1c, 0x21, 0x3f, 0x31, 0xad, 0x1d, 0x4a, 0x44, 0x18, 0xd0, 0x81, - 0x61, 0x1e, 0x19, 0x32, 0x21, 0x53, 0x14, 0x7b, 0x6c, 0xf0, 0x2d, 0xe3, 0xf2, 0x87, 0xff, 0xe5, - 0x74, 0xf1, 0xd1, 0x58, 0x51, 0x59, 0x34, 0x02, 0xad, 0xd7, 0xd3, 0xb5, 0xe5, 0x9d, 0x9d, 0xfa, - 0x9a, 0x54, 0xa4, 0x22, 0x5f, 0x31, 0x89, 0xcd, 0x63, 0xc3, 0x16, 0xff, 0x51, 0x02, 0x8a, 0xbe, - 0x8e, 0xe2, 0xec, 0xc8, 0x2a, 0x64, 0x5f, 0xf7, 0xb0, 0xa5, 0xbf, 0x41, 0x37, 0x02, 0x67, 0x24, - 0x66, 0xe7, 0x73, 0x98, 0x09, 0x69, 0x20, 0xf9, 0x93, 0x69, 0x20, 0x7b, 0xe4, 0x37, 0x1e, 0xdd, - 0x87, 0x59, 0xe7, 0xd8, 0x90, 0x59, 0x3c, 0x1e, 0x8b, 0xe1, 0x70, 0xc3, 0x0e, 0x0a, 0x0e, 0xd1, - 0x07, 0xa1, 0xd3, 0xf8, 0x0d, 0x5b, 0xfc, 0x23, 0x01, 0x10, 0x55, 0x54, 0x9d, 0xed, 0xe9, 0x7f, - 0x5d, 0xc6, 0xd3, 0x5d, 0x28, 0xd2, 0x08, 0x47, 0x59, 0xdf, 0x93, 0x3b, 0xba, 0x6d, 0xeb, 0x46, - 0x8b, 0x0f, 0xa8, 0x3c, 0xa5, 0xd7, 0xf7, 0x36, 0x19, 0x55, 0xfc, 0xab, 0x30, 0x17, 0x6a, 0x40, - 0x9c, 0x9d, 0x7d, 0x1d, 0x66, 0xf6, 0xcc, 0x9e, 0xa1, 0xc9, 0xec, 0x8d, 0x07, 0xdf, 0x0e, 0xcc, - 0x52, 0x1a, 0x7b, 0x9e, 0xf8, 0x3f, 0x12, 0x30, 0x2f, 0x61, 0xdb, 0x6c, 0x1f, 0xe2, 0xf8, 0x55, - 0x58, 0x03, 0xfe, 0xae, 0x45, 0x7e, 0x23, 0x4d, 0x66, 0x18, 0x33, 0x5b, 0xe6, 0xc2, 0x7b, 0xea, - 0x37, 0x47, 0x8f, 0xd8, 0xc1, 0x5d, 0x74, 0xbe, 0x27, 0x97, 0x0a, 0xed, 0xc9, 0x99, 0x50, 0xd0, - 0x5b, 0x86, 0x49, 0x6c, 0x9a, 0x8d, 0x5f, 0x1b, 0xbd, 0x8e, 0x8b, 0x54, 0x96, 0x47, 0x55, 0xb2, - 0xce, 0x58, 0x1a, 0xf8, 0xf5, 0x56, 0xaf, 0x43, 0x7d, 0xe7, 0xf2, 0x65, 0x52, 0xdf, 0xb3, 0xd3, - 0xc5, 0x7c, 0xe8, 0x9e, 0x2d, 0xe5, 0x75, 0xef, 0x9a, 0x48, 0x17, 0xbf, 0x0d, 0x97, 0xfa, 0x94, - 0x1d, 0xa7, 0xc7, 0xf3, 0x6f, 0x92, 0x70, 0x35, 0x2c, 0x3e, 0x6e, 0xfc, 0xf1, 0x75, 0xef, 0xd0, - 0x1a, 0xe4, 0x3a, 0xba, 0xf1, 0x66, 0x5b, 0x8b, 0x33, 0x1d, 0xdd, 0xf0, 0xb7, 0x71, 0x23, 0x86, - 0xc6, 0xd4, 0x57, 0x3a, 0x34, 0x14, 0x58, 0x88, 0xea, 0xbb, 0x38, 0xc7, 0xc7, 0xaf, 0x09, 0x30, - 0x13, 0xf7, 0x9e, 0xd9, 0x9b, 0xc5, 0x9b, 0x89, 0x4d, 0xc8, 0x7d, 0x05, 0x9b, 0x6c, 0xbf, 0x23, - 0x00, 0x6a, 0x5a, 0x3d, 0x83, 0x80, 0xda, 0x67, 0x66, 0x2b, 0xce, 0x66, 0xce, 0xc3, 0xa4, 0x6e, - 0x68, 0xf8, 0x98, 0x36, 0x33, 0x25, 0xb1, 0x8b, 0xd0, 0xab, 0xc3, 0xe4, 0x58, 0xaf, 0x0e, 0xc5, - 0xcf, 0x61, 0x2e, 0x54, 0xc5, 0x38, 0xdb, 0xff, 0x8f, 0x13, 0x30, 0xc7, 0x1b, 0x12, 0xfb, 0xf6, - 0xe2, 0x37, 0x61, 0xb2, 0x4d, 0x64, 0x8e, 0xe8, 0x67, 0xfa, 0x4c, 0xb7, 0x9f, 0x69, 0x61, 0xf4, - 0x73, 0x00, 0x5d, 0x0b, 0x1f, 0xca, 0x8c, 0x35, 0x39, 0x16, 0x6b, 0x86, 0x70, 0x50, 0x02, 0xfa, - 0x05, 0x28, 0x90, 0xf9, 0xdc, 0xb5, 0xcc, 0xae, 0x69, 0x13, 0x97, 0xc5, 0x1e, 0x0f, 0xe5, 0xcc, - 0x9e, 0x9d, 0x2e, 0xe6, 0x36, 0x75, 0x63, 0x9b, 0x33, 0x36, 0x1b, 0x12, 0x31, 0x0c, 0xde, 0xa5, - 0x2d, 0xfe, 0x27, 0x01, 0xe6, 0xbf, 0xb2, 0xad, 0xd8, 0xff, 0x1f, 0xba, 0x12, 0x5f, 0x42, 0x91, - 0xfe, 0xa8, 0x1b, 0x7b, 0x66, 0x9c, 0x9b, 0xe2, 0xdf, 0x13, 0x60, 0x36, 0x20, 0x38, 0x4e, 0xff, - 0xe4, 0x8d, 0xf4, 0x24, 0xfe, 0x22, 0xf1, 0x58, 0x82, 0x83, 0x3c, 0xce, 0x29, 0xf4, 0x07, 0x09, - 0xb8, 0x5c, 0x61, 0xaf, 0x90, 0xdd, 0x98, 0x8a, 0x38, 0x47, 0x46, 0x09, 0xa6, 0x0f, 0xb1, 0x65, - 0xeb, 0x26, 0x5b, 0x3d, 0x73, 0x92, 0x7b, 0x89, 0x16, 0x20, 0x6d, 0x1b, 0x4a, 0xd7, 0xde, 0x37, - 0xdd, 0x77, 0x67, 0xde, 0xb5, 0x17, 0xff, 0x31, 0xf9, 0xe6, 0xf1, 0x1f, 0x53, 0xa3, 0xe3, 0x3f, - 0xa6, 0x7f, 0x82, 0xf8, 0x0f, 0xfe, 0xa2, 0xea, 0x3f, 0x0a, 0x70, 0x65, 0x40, 0x73, 0x71, 0x8e, - 0x96, 0xef, 0x42, 0x56, 0xe5, 0x82, 0x89, 0xbd, 0x65, 0x6f, 0xe1, 0xea, 0xa4, 0xd8, 0x1b, 0xc2, - 0x8e, 0xb3, 0xd3, 0x45, 0x70, 0xab, 0x5a, 0x5f, 0xe3, 0xca, 0x21, 0xbf, 0x35, 0xf1, 0x57, 0x72, - 0x50, 0xa8, 0x1e, 0xb3, 0x1d, 0xe8, 0x06, 0x5b, 0xe5, 0xd1, 0x13, 0x48, 0x77, 0x2d, 0xf3, 0x50, - 0x77, 0x9b, 0x91, 0x0f, 0xbd, 0xfc, 0x77, 0x9b, 0xd1, 0xc7, 0xb5, 0xcd, 0x39, 0x24, 0x8f, 0x17, - 0x35, 0x21, 0xf3, 0xcc, 0x54, 0x95, 0xf6, 0x13, 0xbd, 0xed, 0x8e, 0xfc, 0xf7, 0xcf, 0x17, 0xb4, - 0xec, 0xf1, 0x6c, 0x2b, 0xce, 0xbe, 0xdb, 0x09, 0x1e, 0x11, 0xd5, 0x21, 0x5d, 0x73, 0x9c, 0x2e, - 0xb9, 0xc9, 0x6d, 0xc7, 0x9d, 0x31, 0x84, 0x12, 0x16, 0x37, 0x56, 0xd4, 0x65, 0x47, 0x4d, 0x98, - 0x7d, 0x4a, 0x4f, 0x38, 0x55, 0xda, 0x66, 0x4f, 0xab, 0x98, 0xc6, 0x9e, 0xde, 0xe2, 0x76, 0xf7, - 0xf6, 0x18, 0x32, 0x9f, 0x56, 0x1a, 0xd2, 0xa0, 0x00, 0xb4, 0x0a, 0xe9, 0xc6, 0x23, 0x2e, 0x8c, - 0xb9, 0x65, 0xb7, 0xc6, 0x10, 0xd6, 0x78, 0x24, 0x79, 0x6c, 0x68, 0x1d, 0xb2, 0xab, 0x5f, 0xf4, - 0x2c, 0xcc, 0xa5, 0x4c, 0x0d, 0x8d, 0x3c, 0xe8, 0x97, 0x42, 0xb9, 0xa4, 0x20, 0x33, 0x6a, 0x40, - 0xfe, 0x95, 0x69, 0x1d, 0xb4, 0x4d, 0xc5, 0x6d, 0xe1, 0x34, 0x15, 0xf7, 0x8d, 0x31, 0xc4, 0xb9, - 0x8c, 0x52, 0x9f, 0x08, 0xf4, 0x6d, 0x28, 0x90, 0xce, 0x68, 0x2a, 0xbb, 0x6d, 0xb7, 0x92, 0x69, - 0x2a, 0xf5, 0xdd, 0x31, 0xa4, 0x7a, 0x9c, 0xee, 0x2b, 0x90, 0x3e, 0x51, 0x0b, 0x12, 0xe4, 0x42, - 0x83, 0x00, 0x21, 0x48, 0x75, 0x49, 0x7f, 0x0b, 0x34, 0x36, 0x88, 0xfe, 0x46, 0xef, 0xc1, 0xb4, - 0x61, 0x6a, 0xd8, 0x9d, 0x21, 0xb9, 0xf2, 0xfc, 0xd9, 0xe9, 0xe2, 0xd4, 0x96, 0xa9, 0x31, 0x87, - 0x84, 0xff, 0x92, 0xa6, 0x48, 0xa1, 0xba, 0xb6, 0xb0, 0x04, 0x29, 0xd2, 0xef, 0xc4, 0x30, 0xed, - 0x2a, 0x36, 0xde, 0xb1, 0x74, 0x2e, 0xcd, 0xbd, 0x5c, 0xf8, 0x67, 0x09, 0x48, 0x34, 0x1e, 0x11, - 0x97, 0x7b, 0xb7, 0xa7, 0x1e, 0x60, 0x87, 0xdf, 0xe7, 0x57, 0xd4, 0x15, 0xb7, 0xf0, 0x9e, 0xce, - 0x3c, 0xa3, 0x8c, 0xc4, 0xaf, 0xd0, 0x3b, 0x00, 0x8a, 0xaa, 0x62, 0xdb, 0x96, 0xdd, 0x93, 0x6f, - 0x19, 0x29, 0xc3, 0x28, 0x1b, 0xf8, 0x84, 0xb0, 0xd9, 0x58, 0xb5, 0xb0, 0xe3, 0x06, 0x36, 0xb1, - 0x2b, 0xc2, 0xe6, 0xe0, 0x4e, 0x57, 0x76, 0xcc, 0x03, 0x6c, 0xd0, 0x71, 0x92, 0x21, 0xa6, 0xa6, - 0xd3, 0x6d, 0x12, 0x02, 0xb1, 0x92, 0xd8, 0xd0, 0x7c, 0x93, 0x96, 0x91, 0xbc, 0x6b, 0x22, 0xd2, - 0xc2, 0x2d, 0x9d, 0x1f, 0xe1, 0xca, 0x48, 0xfc, 0x8a, 0x68, 0x49, 0xe9, 0x39, 0xfb, 0xb4, 0x27, - 0x32, 0x12, 0xfd, 0x8d, 0x6e, 0x43, 0x81, 0xc5, 0x42, 0xca, 0xd8, 0x50, 0x65, 0x6a, 0x5c, 0x33, - 0xf4, 0x76, 0x8e, 0x91, 0xab, 0x86, 0x4a, 0x4c, 0x29, 0x7a, 0x04, 0x9c, 0x20, 0x1f, 0x74, 0x6c, - 0xa2, 0x53, 0x20, 0xa5, 0xca, 0x85, 0xb3, 0xd3, 0xc5, 0x6c, 0x83, 0xde, 0xd8, 0xd8, 0x6c, 0xd4, - 0xd7, 0xa4, 0x2c, 0x2b, 0xb5, 0xd1, 0xb1, 0xeb, 0xda, 0xc2, 0x6f, 0x08, 0x90, 0x7c, 0x5a, 0x69, - 0x5c, 0x58, 0x65, 0x6e, 0x45, 0x93, 0x81, 0x8a, 0xde, 0x81, 0xc2, 0xae, 0xde, 0x6e, 0xeb, 0x46, - 0x8b, 0x78, 0x41, 0xdf, 0xc5, 0xaa, 0xab, 0xb0, 0x3c, 0x27, 0x6f, 0x33, 0x2a, 0x5a, 0x82, 0xac, - 0x6a, 0x61, 0x0d, 0x1b, 0x8e, 0xae, 0xb4, 0x6d, 0xae, 0xb9, 0x20, 0x69, 0xe1, 0x97, 0x05, 0x98, - 0xa4, 0x33, 0x00, 0xbd, 0x0d, 0x19, 0xd5, 0x34, 0x1c, 0x45, 0x37, 0xb8, 0x29, 0xcb, 0x48, 0x3e, - 0x61, 0x68, 0xf5, 0xae, 0xc3, 0x8c, 0xa2, 0xaa, 0x66, 0xcf, 0x70, 0x64, 0x43, 0xe9, 0x60, 0x5e, - 0xcd, 0x2c, 0xa7, 0x6d, 0x29, 0x1d, 0x8c, 0x16, 0xc1, 0xbd, 0xf4, 0x0e, 0x34, 0x66, 0x24, 0xe0, - 0xa4, 0x0d, 0x7c, 0xb2, 0xf0, 0xc7, 0x02, 0xa4, 0xdd, 0x39, 0x43, 0xaa, 0xd1, 0xc2, 0x06, 0xb6, - 0x14, 0xc7, 0xf4, 0xaa, 0xe1, 0x11, 0xfa, 0x97, 0xca, 0x8c, 0xbf, 0x54, 0xce, 0xc3, 0xa4, 0x43, - 0xa6, 0x05, 0xaf, 0x01, 0xbb, 0xa0, 0xdb, 0xd1, 0x6d, 0xa5, 0xc5, 0x76, 0xe3, 0x32, 0x12, 0xbb, - 0x20, 0x8d, 0xe1, 0x21, 0xb5, 0x4c, 0x23, 0xfc, 0x8a, 0xd4, 0x94, 0x05, 0x7e, 0xee, 0xe2, 0x96, - 0x6e, 0xd0, 0xb1, 0x94, 0x94, 0x80, 0x92, 0xca, 0x84, 0x82, 0xde, 0x82, 0x0c, 0x2b, 0x80, 0x0d, - 0x8d, 0x0e, 0xa8, 0xa4, 0x94, 0xa6, 0x84, 0xaa, 0xa1, 0x2d, 0x60, 0xc8, 0x78, 0x93, 0x93, 0x74, - 0x5b, 0xcf, 0xf6, 0x14, 0x49, 0x7f, 0xa3, 0xf7, 0x61, 0xfe, 0x75, 0x4f, 0x69, 0xeb, 0x7b, 0x74, - 0xa3, 0x8d, 0x14, 0x63, 0x3a, 0x63, 0x2d, 0x41, 0xde, 0x3d, 0x2a, 0x81, 0xaa, 0xce, 0x9d, 0xcb, - 0x49, 0x7f, 0x2e, 0x8b, 0xbf, 0x2f, 0xc0, 0x2c, 0x8b, 0xea, 0x61, 0xc1, 0xa8, 0xf1, 0xf9, 0x21, - 0x1f, 0x43, 0x46, 0x53, 0x1c, 0x85, 0x1d, 0xd1, 0x4c, 0x8c, 0x3c, 0xa2, 0xe9, 0x1d, 0x25, 0x50, - 0x1c, 0x85, 0x1e, 0xd3, 0x44, 0x90, 0x22, 0xbf, 0xd9, 0x69, 0x56, 0x89, 0xfe, 0x16, 0x3f, 0x03, - 0x14, 0xac, 0x68, 0x9c, 0x1e, 0xd9, 0x3d, 0xb8, 0x44, 0x74, 0x5d, 0x35, 0x54, 0xeb, 0xa4, 0xeb, - 0xe8, 0xa6, 0xf1, 0x9c, 0xfe, 0xb5, 0x51, 0x31, 0xf0, 0x5e, 0x8a, 0xbe, 0x8e, 0x12, 0xff, 0x70, - 0x0a, 0x72, 0xd5, 0xe3, 0xae, 0x69, 0xc5, 0xba, 0x8b, 0x55, 0x86, 0x69, 0x0e, 0xf4, 0x47, 0xbc, - 0x17, 0xee, 0x33, 0xe6, 0xee, 0x2b, 0x57, 0xce, 0x88, 0xca, 0x00, 0x2c, 0x60, 0x94, 0x06, 0x05, - 0x25, 0x2f, 0xf0, 0xa6, 0x8c, 0xb2, 0x11, 0x2a, 0xda, 0x82, 0x6c, 0xe7, 0x50, 0x55, 0xe5, 0x3d, - 0xbd, 0xed, 0xf0, 0xb8, 0xbb, 0xe8, 0x10, 0xf1, 0xcd, 0x97, 0x95, 0xca, 0x13, 0x5a, 0x88, 0x85, - 0xc0, 0xf9, 0xd7, 0x12, 0x10, 0x09, 0xec, 0x37, 0x7a, 0x17, 0xf8, 0x91, 0x1a, 0xd9, 0x76, 0x4f, - 0xc9, 0x95, 0x73, 0x67, 0xa7, 0x8b, 0x19, 0x89, 0x52, 0x1b, 0x8d, 0xa6, 0x94, 0x61, 0x05, 0x1a, - 0xb6, 0x83, 0x6e, 0x40, 0xce, 0xec, 0xe8, 0x8e, 0xec, 0x3a, 0x49, 0xdc, 0xa3, 0x9c, 0x21, 0x44, - 0xd7, 0x89, 0x42, 0x4d, 0xb8, 0x83, 0x0d, 0x3a, 0xda, 0x49, 0x3b, 0xe5, 0x5d, 0xb6, 0xf9, 0xe8, - 0xb0, 0x19, 0x2d, 0x9b, 0x5d, 0x47, 0xef, 0xe8, 0x5f, 0xd0, 0x37, 0xd3, 0xfc, 0xa5, 0xd1, 0x0d, - 0x56, 0x9c, 0xb4, 0xaf, 0x4c, 0x77, 0x25, 0x79, 0xd9, 0xe7, 0x81, 0xa2, 0xe8, 0x6f, 0x0a, 0x70, - 0x99, 0x2b, 0x52, 0xde, 0xa5, 0x31, 0xee, 0x4a, 0x5b, 0x77, 0x4e, 0xe4, 0x83, 0xc3, 0x52, 0x9a, - 0xfa, 0xad, 0x3f, 0x1b, 0xd9, 0x21, 0x81, 0x71, 0xb0, 0xec, 0x76, 0xcb, 0xc9, 0x33, 0xce, 0xbc, - 0x71, 0x58, 0x35, 0x1c, 0xeb, 0xa4, 0x7c, 0xe5, 0xec, 0x74, 0x71, 0x6e, 0xf0, 0xee, 0x4b, 0x69, - 0xce, 0x1e, 0x64, 0x41, 0x35, 0x00, 0xec, 0x8d, 0x43, 0xba, 0x62, 0x44, 0xfb, 0x1f, 0x91, 0x03, - 0x56, 0x0a, 0xf0, 0xa2, 0xbb, 0x50, 0xe4, 0x27, 0x5b, 0xf6, 0xf4, 0x36, 0x96, 0x6d, 0xfd, 0x0b, - 0x4c, 0xd7, 0x96, 0xa4, 0x94, 0x67, 0x74, 0x22, 0xa2, 0xa1, 0x7f, 0x81, 0x17, 0xbe, 0x0b, 0xa5, - 0x61, 0xb5, 0x0f, 0x4e, 0x81, 0x0c, 0x7b, 0x23, 0xfb, 0x51, 0x78, 0x3b, 0x66, 0x8c, 0xa1, 0xca, - 0xb7, 0x64, 0x3e, 0x4e, 0x7c, 0x24, 0x88, 0xff, 0x24, 0x01, 0xb9, 0x72, 0xaf, 0x7d, 0xf0, 0xbc, - 0xdb, 0xe8, 0x75, 0x3a, 0x8a, 0x75, 0x42, 0xcc, 0x20, 0x33, 0x14, 0xa4, 0x82, 0x02, 0x33, 0x83, - 0xd4, 0x12, 0xe8, 0x5f, 0x60, 0xb2, 0x38, 0x05, 0x4f, 0x65, 0xb3, 0x18, 0x7e, 0xda, 0x86, 0xc0, - 0x51, 0x6b, 0xf3, 0xc8, 0x46, 0x1f, 0x41, 0x29, 0x50, 0x90, 0xee, 0x9d, 0xc8, 0xd8, 0x70, 0x2c, - 0x1d, 0xb3, 0xfd, 0xbf, 0xa4, 0x14, 0x88, 0x97, 0xa9, 0x93, 0xdb, 0x55, 0x76, 0x17, 0x35, 0x61, - 0x86, 0x14, 0x3c, 0x91, 0xe9, 0x12, 0xe2, 0xee, 0xcf, 0x3e, 0x88, 0x68, 0x56, 0xa8, 0xde, 0xcb, - 0x54, 0x3f, 0x15, 0xca, 0x43, 0x7f, 0x4a, 0x59, 0xec, 0x53, 0x16, 0x3e, 0x85, 0x62, 0x7f, 0x81, - 0xa0, 0x2e, 0x53, 0x4c, 0x97, 0xf3, 0x41, 0x5d, 0x26, 0x03, 0x7a, 0x5a, 0x4f, 0xa5, 0x53, 0xc5, - 0x49, 0xf1, 0x2f, 0x92, 0x90, 0x77, 0x87, 0x59, 0x9c, 0x40, 0xa7, 0x0c, 0x93, 0x64, 0x50, 0xb8, - 0x31, 0x1e, 0xb7, 0x47, 0x8c, 0x6e, 0x1e, 0x35, 0x4e, 0x06, 0x8b, 0x0b, 0x92, 0x29, 0x6b, 0x1c, - 0x06, 0x67, 0xe1, 0x97, 0x13, 0x90, 0xa2, 0xd8, 0xe2, 0x01, 0xa4, 0xe8, 0x42, 0x21, 0x8c, 0xb3, - 0x50, 0xd0, 0xa2, 0xde, 0x72, 0x96, 0x08, 0xb8, 0xa6, 0xc4, 0xe7, 0xdb, 0x57, 0x3e, 0x78, 0xf0, - 0x90, 0x1a, 0x9b, 0x19, 0x89, 0x5f, 0xa1, 0x32, 0x0d, 0x3b, 0x32, 0x2d, 0x07, 0x6b, 0xdc, 0xa7, - 0x5f, 0x3a, 0xaf, 0x7f, 0xdd, 0x45, 0xc9, 0xe5, 0x43, 0x57, 0x21, 0x49, 0xac, 0xd8, 0x34, 0x0b, - 0x52, 0x38, 0x3b, 0x5d, 0x4c, 0x12, 0xfb, 0x45, 0x68, 0x68, 0x05, 0xb2, 0x61, 0x93, 0x41, 0x3c, - 0x38, 0x6a, 0x18, 0x03, 0xd3, 0x1d, 0xda, 0xde, 0xd4, 0x62, 0x78, 0x96, 0xf7, 0xf1, 0x5f, 0xa6, - 0x20, 0x57, 0xef, 0xc4, 0xbd, 0xa4, 0xac, 0x86, 0x7b, 0x38, 0x0a, 0x08, 0x85, 0x1e, 0x1a, 0xd1, - 0xc1, 0xa1, 0x15, 0x3c, 0x79, 0xb1, 0x15, 0xbc, 0x4e, 0x3c, 0x65, 0x9e, 0x66, 0x21, 0x39, 0x04, - 0xf3, 0x84, 0x9f, 0x4f, 0xfd, 0x14, 0x89, 0xf0, 0xf8, 0xe7, 0x28, 0x68, 0xa0, 0xc9, 0xa7, 0xd4, - 0x21, 0x67, 0xa3, 0x6c, 0x6a, 0xfc, 0x51, 0x36, 0x8d, 0x0d, 0x8d, 0x2e, 0x6a, 0x61, 0x8b, 0x3a, - 0xfd, 0xe6, 0x16, 0x75, 0xc1, 0xe1, 0x83, 0xf5, 0x63, 0x48, 0x6a, 0xba, 0xdb, 0x39, 0xe3, 0x2f, - 0xd5, 0x84, 0xe9, 0x9c, 0x51, 0x9b, 0x0a, 0x8e, 0x5a, 0x36, 0x4a, 0x16, 0xea, 0x00, 0xbe, 0x6e, - 0xd0, 0x12, 0x4c, 0x99, 0x6d, 0xcd, 0x3d, 0x48, 0x92, 0x2b, 0x67, 0xce, 0x4e, 0x17, 0x27, 0x9f, - 0xb7, 0xb5, 0xfa, 0x9a, 0x34, 0x69, 0xb6, 0xb5, 0xba, 0x46, 0x73, 0x5c, 0xe0, 0x23, 0xd9, 0x8b, - 0x32, 0x9b, 0x91, 0xa6, 0x0d, 0x7c, 0xb4, 0x86, 0x6d, 0x95, 0x0f, 0xb8, 0xdf, 0x16, 0x20, 0xef, - 0xea, 0x3e, 0x5e, 0xa3, 0x92, 0xd6, 0x3b, 0x7c, 0x92, 0x25, 0x2f, 0x36, 0xc9, 0x5c, 0x3e, 0x7e, - 0xbc, 0xf6, 0x57, 0x05, 0x1e, 0x37, 0xdc, 0x50, 0x15, 0x87, 0x38, 0x15, 0x31, 0x4e, 0x8c, 0x7b, - 0x50, 0xb4, 0x14, 0x43, 0x33, 0x3b, 0xfa, 0x17, 0x98, 0x6d, 0x84, 0xda, 0xfc, 0xad, 0x65, 0xc1, - 0xa3, 0xd3, 0x5d, 0x3f, 0x5b, 0xfc, 0xe3, 0x04, 0x8f, 0x31, 0xf6, 0xaa, 0x11, 0xa7, 0xba, 0xbe, - 0x03, 0xb3, 0xfd, 0x09, 0x47, 0xdc, 0xd9, 0xfa, 0x5e, 0x84, 0xbc, 0xa8, 0x8a, 0xb0, 0x58, 0x41, - 0x37, 0x70, 0xbd, 0x2f, 0xf9, 0x88, 0x8d, 0x2a, 0x90, 0x0d, 0xe6, 0x31, 0x49, 0x8e, 0x9d, 0xc7, - 0x04, 0x2c, 0x2f, 0x7b, 0xc9, 0xc2, 0xcf, 0xc3, 0x24, 0xbd, 0xfd, 0x06, 0x26, 0x9a, 0xf7, 0xe6, - 0x9f, 0x27, 0xe0, 0x26, 0xad, 0xfd, 0x4b, 0x6c, 0xe9, 0x7b, 0x27, 0xdb, 0x96, 0xe9, 0x60, 0xd5, - 0xc1, 0x9a, 0x7f, 0x0a, 0x24, 0x56, 0xbb, 0x97, 0xe9, 0xba, 0x0f, 0xb8, 0x50, 0xbc, 0x98, 0xc7, - 0x85, 0x36, 0xa0, 0xc0, 0x23, 0x03, 0x94, 0xb6, 0x7e, 0x88, 0x65, 0xc5, 0xb9, 0xc8, 0xea, 0x96, - 0x63, 0xbc, 0xab, 0x84, 0x75, 0xd5, 0x41, 0x1a, 0x64, 0xb8, 0x30, 0x5d, 0xe3, 0xe9, 0x77, 0x9e, - 0xfe, 0x64, 0x1b, 0x8a, 0x69, 0x16, 0x9e, 0x50, 0x5f, 0x93, 0xd2, 0x4c, 0x72, 0x5d, 0x13, 0xff, - 0xb3, 0x00, 0xb7, 0xce, 0x51, 0x71, 0x9c, 0x43, 0x77, 0x01, 0xd2, 0x87, 0xe4, 0x41, 0x3a, 0xd7, - 0x71, 0x5a, 0xf2, 0xae, 0xd1, 0x26, 0xe4, 0xf6, 0x14, 0xbd, 0xed, 0x0f, 0xe9, 0xe1, 0xe1, 0x85, - 0xd1, 0x91, 0xae, 0x33, 0x8c, 0x9d, 0x8d, 0x61, 0xf1, 0x37, 0x13, 0x30, 0xbb, 0xaa, 0x69, 0x8d, - 0x06, 0xb7, 0x81, 0xf1, 0x8d, 0x14, 0x17, 0x64, 0x26, 0x7c, 0x90, 0x89, 0xde, 0x03, 0xa4, 0xe9, - 0x36, 0x4b, 0xf3, 0x61, 0xef, 0x2b, 0x9a, 0x79, 0xe4, 0x87, 0x55, 0xcc, 0xba, 0x77, 0x1a, 0xee, - 0x0d, 0xd4, 0x00, 0x8a, 0x76, 0x64, 0xdb, 0x51, 0xbc, 0xf7, 0x46, 0xb7, 0xc6, 0x3a, 0xae, 0xc5, - 0x60, 0x90, 0x77, 0x29, 0x65, 0x88, 0x1c, 0xfa, 0x93, 0xf8, 0xed, 0x3a, 0x69, 0xba, 0x23, 0x2b, - 0xb6, 0x7b, 0x36, 0x87, 0x25, 0x18, 0xc9, 0x33, 0xfa, 0xaa, 0xcd, 0x8e, 0xdc, 0xb0, 0xc3, 0x04, - 0xbe, 0x6a, 0xe2, 0x84, 0xc4, 0xff, 0x40, 0x80, 0xbc, 0x84, 0xf7, 0x2c, 0x6c, 0xc7, 0xba, 0x29, - 0xf0, 0x04, 0x66, 0x2c, 0x26, 0x55, 0xde, 0xb3, 0xcc, 0xce, 0x45, 0xe6, 0x55, 0x96, 0x33, 0x3e, - 0xb1, 0xcc, 0x0e, 0x37, 0x2c, 0x2f, 0xa1, 0xe0, 0xd5, 0x31, 0xce, 0xc6, 0xff, 0x3e, 0x3d, 0x8a, - 0xcc, 0x04, 0xc7, 0x1d, 0xdf, 0x10, 0xaf, 0x06, 0xe8, 0x8b, 0xaa, 0x60, 0x45, 0xe3, 0x54, 0xc3, - 0x7f, 0x17, 0x20, 0xdf, 0xe8, 0xed, 0xb2, 0x3c, 0x52, 0xf1, 0x69, 0xa0, 0x0a, 0x99, 0x36, 0xde, - 0x73, 0xe4, 0x37, 0x0a, 0x83, 0x4f, 0x13, 0x56, 0x7a, 0x08, 0xe0, 0x29, 0x80, 0x45, 0x8f, 0xaf, - 0x51, 0x39, 0xc9, 0x0b, 0xca, 0xc9, 0x50, 0x5e, 0x42, 0x16, 0xff, 0x69, 0x02, 0x0a, 0x5e, 0x33, - 0xe3, 0xb4, 0x92, 0xaf, 0x42, 0xd6, 0x21, 0x79, 0x11, 0xeb, 0x30, 0xcb, 0x43, 0x3a, 0xa2, 0x2d, - 0xc4, 0x32, 0xcc, 0x51, 0xc7, 0x45, 0x56, 0xba, 0xdd, 0xb6, 0xee, 0xc2, 0x5d, 0x6a, 0x7f, 0x52, - 0xd2, 0x2c, 0xbd, 0xb5, 0xca, 0xee, 0x50, 0xa0, 0x4b, 0xc6, 0xdc, 0x9e, 0x85, 0xf1, 0x17, 0x58, - 0xa6, 0xc8, 0xeb, 0x22, 0x21, 0x2b, 0x59, 0xc6, 0xd8, 0x20, 0x7c, 0x7c, 0xcc, 0xbd, 0x82, 0x59, - 0xaa, 0xd3, 0xb8, 0x8f, 0xdd, 0x8a, 0x7f, 0x2f, 0x01, 0x28, 0x28, 0xf9, 0xab, 0xeb, 0x8b, 0x44, - 0x7c, 0x7d, 0xf1, 0x2e, 0x20, 0x16, 0xb4, 0x68, 0xcb, 0x5d, 0x6c, 0xc9, 0x36, 0x56, 0x4d, 0x9e, - 0xe3, 0x48, 0x90, 0x8a, 0xfc, 0xce, 0x36, 0xb6, 0x1a, 0x94, 0x8e, 0x1e, 0x03, 0xf8, 0x1e, 0x19, - 0x5f, 0x30, 0x46, 0x3a, 0x64, 0x52, 0xc6, 0x73, 0xc5, 0xc4, 0xef, 0x2d, 0xc0, 0x0c, 0xd7, 0xe1, - 0x8e, 0xa1, 0x9b, 0x06, 0x7a, 0x00, 0xc9, 0x16, 0xdf, 0xfa, 0xcf, 0x46, 0x6e, 0xd3, 0xf9, 0x49, - 0xdc, 0x6a, 0x13, 0x12, 0x29, 0x4b, 0x58, 0xba, 0x3d, 0x27, 0xc2, 0x31, 0xf2, 0x03, 0xb1, 0x83, - 0x2c, 0xdd, 0x9e, 0x83, 0x1a, 0x50, 0x50, 0xfd, 0x24, 0x55, 0x32, 0x61, 0x4f, 0x0e, 0x05, 0x51, - 0x91, 0x69, 0xc1, 0x6a, 0x13, 0x52, 0x5e, 0x0d, 0xdd, 0x40, 0x95, 0x60, 0x6e, 0xa4, 0xd4, 0x40, - 0x94, 0x97, 0x7f, 0xbe, 0x37, 0x9c, 0x97, 0xa9, 0x36, 0x11, 0x48, 0xa1, 0x84, 0x3e, 0x86, 0x29, - 0x8d, 0x66, 0xe1, 0xe1, 0x23, 0x3a, 0x6a, 0xd0, 0x85, 0x92, 0x1d, 0xd5, 0x26, 0x24, 0xce, 0x81, - 0xd6, 0x61, 0x86, 0xfd, 0x62, 0x6e, 0x0a, 0x47, 0x96, 0xb7, 0x86, 0x4b, 0x08, 0x2c, 0x04, 0xb5, - 0x09, 0x29, 0xab, 0xf9, 0x54, 0xf4, 0x4d, 0x48, 0xd9, 0xaa, 0xe2, 0x62, 0xcb, 0x6b, 0x43, 0x52, - 0x6a, 0xf8, 0xcc, 0xb4, 0x34, 0x7a, 0xcc, 0xd2, 0x32, 0x3a, 0xc7, 0xee, 0x36, 0x5f, 0x54, 0xf5, - 0x43, 0x47, 0xb7, 0x49, 0xf5, 0x31, 0x25, 0xa0, 0xa7, 0x90, 0x55, 0x88, 0xbf, 0x27, 0xd3, 0xc3, - 0x8f, 0x74, 0x5f, 0x2f, 0xfa, 0xe5, 0xfa, 0xc0, 0x61, 0xd5, 0x1a, 0x3d, 0x1f, 0xee, 0x12, 0x7d, - 0x41, 0x1d, 0x6c, 0xb5, 0x70, 0x29, 0x3b, 0x5a, 0x50, 0x30, 0xb6, 0xcb, 0x13, 0x44, 0x89, 0xc4, - 0xef, 0xf3, 0x4e, 0x1e, 0xd3, 0x46, 0xcd, 0x0c, 0x7d, 0x9d, 0x1b, 0x71, 0x6c, 0xa7, 0x36, 0x21, - 0xcd, 0xec, 0x07, 0xc8, 0x68, 0x19, 0x12, 0x2d, 0xb5, 0x94, 0x1b, 0x3a, 0x43, 0xbc, 0xa3, 0x29, - 0xb5, 0x09, 0x29, 0xd1, 0x52, 0xd1, 0xa7, 0x90, 0x66, 0xe7, 0x0c, 0x8e, 0x8d, 0x52, 0x7e, 0xa8, - 0x9d, 0x08, 0x9f, 0xd6, 0xa8, 0x4d, 0x48, 0xf4, 0x68, 0x03, 0x79, 0xde, 0x36, 0xe4, 0x2d, 0x16, - 0x1c, 0xe7, 0x86, 0xb2, 0x16, 0x87, 0xbe, 0xe2, 0x8e, 0x8a, 0x66, 0xad, 0x51, 0xcf, 0x3f, 0x40, - 0x47, 0xdf, 0x81, 0xf9, 0xb0, 0x44, 0x3e, 0xd2, 0x66, 0x87, 0xbe, 0xae, 0x1d, 0x1a, 0x59, 0x59, - 0x9b, 0x90, 0x90, 0x35, 0x70, 0x13, 0x7d, 0x08, 0x93, 0xac, 0xd7, 0x10, 0x15, 0x19, 0x15, 0xb5, - 0xd1, 0xd7, 0x61, 0xac, 0x3c, 0x19, 0xfc, 0x0e, 0x8f, 0x10, 0x93, 0xdb, 0x66, 0xab, 0x34, 0x37, - 0x74, 0xf0, 0x0f, 0xc6, 0xba, 0x91, 0xc1, 0xef, 0xf8, 0x54, 0xd2, 0xef, 0x16, 0xbb, 0xc3, 0xc3, - 0x8a, 0xe6, 0x87, 0xf6, 0x7b, 0x44, 0xe0, 0x58, 0x8d, 0xc6, 0xee, 0xfb, 0x64, 0x52, 0x35, 0x8b, - 0x65, 0x81, 0x91, 0xe9, 0x9c, 0xba, 0x34, 0xb4, 0x6a, 0x83, 0x69, 0x72, 0x6a, 0xd4, 0x47, 0xf2, - 0xa8, 0xe8, 0x25, 0x14, 0x79, 0xae, 0x06, 0xff, 0x9d, 0xc2, 0x65, 0x2a, 0xef, 0x5e, 0xa4, 0xe9, - 0x8a, 0x8a, 0xc9, 0xa9, 0x4d, 0x48, 0x05, 0x35, 0x7c, 0x07, 0x7d, 0x06, 0xb3, 0x54, 0x9e, 0xac, - 0xfa, 0xe9, 0x35, 0x4a, 0xa5, 0x81, 0x64, 0x0d, 0xc3, 0x33, 0x71, 0xb8, 0x92, 0x8b, 0x6a, 0xdf, - 0x2d, 0x32, 0x8c, 0x75, 0x43, 0x77, 0xa8, 0x95, 0x5d, 0x18, 0x3a, 0x8c, 0xc3, 0x49, 0xfd, 0xc8, - 0x30, 0xd6, 0x19, 0x85, 0x0c, 0x63, 0x87, 0xc7, 0x9c, 0xf1, 0xee, 0x78, 0x7b, 0xe8, 0x30, 0x8e, - 0x0a, 0x4e, 0x23, 0xc3, 0xd8, 0x09, 0xd2, 0xc9, 0x30, 0x66, 0x06, 0xa2, 0x4f, 0xee, 0x3b, 0x43, - 0x87, 0xf1, 0xd0, 0x43, 0xc8, 0x64, 0x18, 0x2b, 0x03, 0x37, 0xd1, 0x1a, 0x00, 0x73, 0x67, 0xe8, - 0xa2, 0x78, 0x6d, 0xe8, 0x62, 0xd0, 0x1f, 0x7b, 0x46, 0x16, 0x83, 0xb6, 0x4b, 0x23, 0x86, 0x8c, - 0x82, 0x25, 0x99, 0xbe, 0x42, 0x2d, 0x2d, 0x0e, 0x35, 0x64, 0x03, 0xaf, 0x3b, 0x89, 0x21, 0x3b, - 0xf2, 0x88, 0x64, 0x55, 0x61, 0x7b, 0xbe, 0xa5, 0xa5, 0xe1, 0x66, 0x39, 0xf8, 0xea, 0x87, 0x9a, - 0x65, 0x4a, 0x40, 0xab, 0x90, 0x21, 0x6b, 0xfe, 0x09, 0x35, 0x43, 0xd7, 0x87, 0xfa, 0xa4, 0x7d, - 0x87, 0x53, 0x6a, 0x13, 0x52, 0xfa, 0x35, 0x27, 0x91, 0xc7, 0xb3, 0xdd, 0xb0, 0x92, 0x38, 0xf4, - 0xf1, 0xa1, 0x9d, 0x53, 0xf2, 0x78, 0xc6, 0x81, 0x54, 0xb8, 0xc4, 0xfa, 0x8a, 0x9f, 0x07, 0xb6, - 0xf8, 0x11, 0xd6, 0xd2, 0x0d, 0x2a, 0x6a, 0xe8, 0xb6, 0x52, 0xe4, 0x31, 0xe5, 0xda, 0x84, 0x34, - 0xa7, 0x0c, 0xde, 0x25, 0x13, 0x9e, 0x2f, 0x3d, 0x6c, 0x33, 0xaa, 0x74, 0x73, 0xe8, 0x84, 0x8f, - 0xd8, 0xc3, 0x23, 0x13, 0x5e, 0x09, 0x90, 0xd9, 0x02, 0xa4, 0xc9, 0xb6, 0xcd, 0x5e, 0xb8, 0xdf, - 0x1a, 0xb1, 0x00, 0xf5, 0xed, 0x02, 0xb0, 0x05, 0x48, 0x6b, 0x30, 0x4e, 0x22, 0x48, 0x6d, 0x63, - 0xc5, 0xe2, 0x66, 0xf6, 0xf6, 0x50, 0x41, 0x03, 0xf9, 0xf3, 0x88, 0x20, 0xd5, 0x23, 0x12, 0x87, - 0xc7, 0x72, 0x93, 0xbc, 0x70, 0x87, 0xf1, 0xce, 0x50, 0x87, 0x27, 0x32, 0x0b, 0x0d, 0x71, 0x78, - 0xac, 0xd0, 0x0d, 0xf4, 0x73, 0x30, 0xcd, 0xe1, 0x5b, 0xe9, 0xee, 0x08, 0x37, 0x36, 0x88, 0xb8, - 0xc9, 0xbc, 0xe6, 0x3c, 0xcc, 0xca, 0x32, 0xd8, 0xc8, 0x9a, 0x77, 0x6f, 0x84, 0x95, 0x1d, 0x40, - 0xae, 0xcc, 0xca, 0xfa, 0x64, 0x62, 0x65, 0xd9, 0x38, 0xe5, 0x6b, 0xdd, 0xfd, 0xa1, 0x56, 0x76, - 0xf0, 0xe8, 0x0b, 0xb1, 0xb2, 0xaf, 0x7d, 0x2a, 0x69, 0x99, 0xcd, 0xe0, 0x53, 0xe9, 0x1b, 0x43, - 0x5b, 0x16, 0xc6, 0x91, 0xa4, 0x65, 0x9c, 0x87, 0x74, 0x1b, 0x73, 0x89, 0x99, 0xa6, 0xdf, 0x1d, - 0x7e, 0xe8, 0xbe, 0x1f, 0x74, 0xd4, 0xdc, 0x8d, 0x4a, 0xa6, 0x61, 0xcf, 0x50, 0x59, 0xfc, 0xd0, - 0x31, 0xd7, 0xd4, 0x7b, 0xa3, 0x0d, 0x55, 0xd4, 0x49, 0x6a, 0xcf, 0x50, 0x85, 0x6e, 0xd2, 0xaa, - 0xb2, 0x73, 0x66, 0x74, 0x7e, 0x2f, 0x8f, 0xc8, 0x0f, 0xd0, 0x77, 0xda, 0x8f, 0x56, 0xd5, 0x23, - 0xfa, 0x53, 0xa8, 0xc7, 0x92, 0x57, 0x94, 0x56, 0x46, 0x4f, 0xa1, 0x70, 0xfa, 0x0c, 0x6f, 0x0a, - 0x71, 0xb2, 0xb7, 0x66, 0xba, 0x1e, 0xc6, 0xfb, 0xa3, 0xd7, 0xcc, 0x7e, 0xd7, 0x82, 0xad, 0x99, - 0xdc, 0xa7, 0xf8, 0xeb, 0x02, 0x2c, 0xb1, 0xba, 0xd1, 0x1d, 0xbd, 0x13, 0xd9, 0xdb, 0x17, 0x0d, - 0x9c, 0x79, 0x78, 0x40, 0x1f, 0xf0, 0xe1, 0xb0, 0xea, 0x9e, 0xb3, 0xcf, 0x5b, 0x9b, 0x90, 0xde, - 0x51, 0x46, 0x95, 0x2b, 0x4f, 0xf3, 0xd7, 0xa2, 0xde, 0x21, 0xcf, 0x42, 0xb1, 0xb8, 0x9e, 0x4a, - 0x5f, 0x29, 0x96, 0xd6, 0x53, 0xe9, 0xab, 0xc5, 0x85, 0xf5, 0x54, 0xfa, 0xad, 0xe2, 0xdb, 0xe2, - 0x5f, 0x5e, 0x85, 0x9c, 0x8b, 0xfc, 0x18, 0x22, 0x7a, 0x18, 0x44, 0x44, 0xd7, 0x86, 0x21, 0x22, - 0x8e, 0x15, 0x39, 0x24, 0x7a, 0x18, 0x84, 0x44, 0xd7, 0x86, 0x41, 0x22, 0x9f, 0x87, 0x60, 0xa2, - 0xe6, 0x30, 0x4c, 0x74, 0x6f, 0x0c, 0x4c, 0xe4, 0x89, 0xea, 0x07, 0x45, 0x6b, 0x83, 0xa0, 0xe8, - 0xe6, 0x68, 0x50, 0xe4, 0x89, 0x0a, 0xa0, 0xa2, 0xc7, 0x7d, 0xa8, 0xe8, 0xfa, 0x08, 0x54, 0xe4, - 0xf1, 0xbb, 0xb0, 0x68, 0x23, 0x12, 0x16, 0xdd, 0x3e, 0x0f, 0x16, 0x79, 0x72, 0x42, 0xb8, 0xe8, - 0x83, 0x10, 0x2e, 0x5a, 0x1c, 0x8a, 0x8b, 0x3c, 0x6e, 0x06, 0x8c, 0x3e, 0xe9, 0x07, 0x46, 0xd7, - 0x47, 0x00, 0x23, 0xbf, 0x05, 0x1c, 0x19, 0xd5, 0xa2, 0x90, 0xd1, 0xad, 0x73, 0x90, 0x91, 0x27, - 0x25, 0x08, 0x8d, 0x6a, 0x51, 0xd0, 0xe8, 0xd6, 0x39, 0xd0, 0xa8, 0x4f, 0x12, 0xc3, 0x46, 0x5b, - 0xd1, 0xd8, 0xe8, 0xce, 0xb9, 0xd8, 0xc8, 0x93, 0x16, 0x06, 0x47, 0x2b, 0x01, 0x70, 0xf4, 0xce, - 0x10, 0x70, 0xe4, 0xb1, 0x12, 0x74, 0xf4, 0xad, 0x01, 0x74, 0x24, 0x8e, 0x42, 0x47, 0x1e, 0xaf, - 0x07, 0x8f, 0x5e, 0x0c, 0x81, 0x47, 0x77, 0xcf, 0x87, 0x47, 0x9e, 0xb0, 0x3e, 0x7c, 0xa4, 0x8c, - 0xc4, 0x47, 0xef, 0x8d, 0x89, 0x8f, 0x3c, 0xe9, 0x51, 0x00, 0xe9, 0xa3, 0x30, 0x40, 0x5a, 0x1a, - 0x0e, 0x90, 0x3c, 0x31, 0x1c, 0x21, 0x6d, 0x44, 0x22, 0xa4, 0xdb, 0xe7, 0x21, 0x24, 0x7f, 0x1e, - 0x04, 0x21, 0xd2, 0x56, 0x34, 0x44, 0xba, 0x73, 0x2e, 0x44, 0xf2, 0xbb, 0x3f, 0x84, 0x91, 0x36, - 0x22, 0x31, 0xd2, 0xed, 0xf3, 0x30, 0x92, 0x5f, 0xb9, 0x20, 0x48, 0x7a, 0x35, 0x14, 0x24, 0xdd, - 0x1f, 0x07, 0x24, 0x79, 0x42, 0x07, 0x50, 0xd2, 0xe7, 0xc3, 0x51, 0xd2, 0x37, 0x2e, 0x90, 0xaf, - 0x30, 0x12, 0x26, 0x7d, 0x6b, 0x00, 0x26, 0x89, 0xa3, 0x60, 0x92, 0x3f, 0x9e, 0x5d, 0x9c, 0xa4, - 0x8c, 0x44, 0x35, 0xef, 0x8d, 0x89, 0x6a, 0xfc, 0xc1, 0x17, 0x01, 0x6b, 0xaa, 0x11, 0xb0, 0xe6, - 0xe6, 0x68, 0x58, 0xe3, 0x9b, 0x73, 0x1f, 0xd7, 0xd4, 0xa2, 0x70, 0xcd, 0xad, 0x73, 0x70, 0x8d, - 0x6f, 0x85, 0x02, 0xc0, 0xe6, 0x71, 0x1f, 0xb0, 0xb9, 0x7e, 0x6e, 0xd4, 0x4f, 0x00, 0xd9, 0x94, - 0x07, 0x91, 0xcd, 0x8d, 0x91, 0xc8, 0xc6, 0x93, 0xe0, 0x43, 0x9b, 0xc7, 0x7d, 0xd0, 0xe6, 0xfa, - 0x08, 0x68, 0xe3, 0x57, 0x80, 0x63, 0x1b, 0x6d, 0x34, 0xb6, 0x59, 0x1e, 0x17, 0xdb, 0x78, 0x82, - 0x23, 0xc1, 0xcd, 0x56, 0x34, 0xb8, 0xb9, 0x33, 0xe6, 0x0b, 0xf9, 0x01, 0x74, 0x53, 0x8b, 0x42, - 0x37, 0xb7, 0xce, 0x41, 0x37, 0xc1, 0x35, 0xc4, 0x83, 0x37, 0xb5, 0x28, 0x78, 0x73, 0xeb, 0x1c, - 0x78, 0xe3, 0x4b, 0x0a, 0xe0, 0x9b, 0xe6, 0x30, 0x7c, 0x73, 0x6f, 0x0c, 0x7c, 0xe3, 0x3b, 0x2f, - 0x7d, 0x00, 0xe7, 0xd3, 0x7e, 0x80, 0x23, 0x8e, 0x02, 0x38, 0xfe, 0x8c, 0x74, 0x11, 0xce, 0x56, - 0x34, 0xc2, 0xb9, 0x73, 0x2e, 0xc2, 0x09, 0x1a, 0xc9, 0x00, 0xc4, 0xd9, 0x88, 0x84, 0x38, 0xb7, - 0xcf, 0x83, 0x38, 0xbe, 0x91, 0x0c, 0x62, 0x9c, 0x4f, 0xfb, 0x31, 0x8e, 0x38, 0x0a, 0xe3, 0xf8, - 0x8d, 0x73, 0x41, 0x4e, 0x2d, 0x0a, 0xe4, 0xdc, 0x3a, 0x07, 0xe4, 0xf8, 0x9d, 0x17, 0x40, 0x39, - 0xca, 0x48, 0x94, 0xf3, 0xde, 0x98, 0x28, 0xa7, 0xcf, 0x70, 0x85, 0x61, 0x4e, 0x2d, 0x0a, 0xe6, - 0xdc, 0x3a, 0x07, 0xe6, 0x04, 0x2a, 0xeb, 0xe3, 0x9c, 0xad, 0x68, 0x9c, 0x73, 0xe7, 0x5c, 0x9c, - 0xd3, 0x37, 0x9b, 0x5c, 0xa0, 0xb3, 0x11, 0x09, 0x74, 0x6e, 0x9f, 0x07, 0x74, 0xfa, 0x16, 0x3e, - 0xee, 0x1c, 0xfc, 0xca, 0xf8, 0x48, 0xe7, 0xa3, 0x8b, 0x23, 0x1d, 0xef, 0x99, 0xb1, 0x40, 0x9d, - 0xf5, 0x54, 0xfa, 0xed, 0xe2, 0x3b, 0xe2, 0xdf, 0x9a, 0x86, 0xa9, 0x9a, 0x17, 0xe7, 0xe2, 0xd7, - 0x52, 0x78, 0x93, 0xbc, 0x48, 0x68, 0x8d, 0xcc, 0x58, 0x6a, 0xf7, 0xce, 0x4f, 0x75, 0x37, 0x98, - 0x8c, 0x8d, 0xb3, 0xbe, 0xc1, 0x01, 0x65, 0xf4, 0x01, 0xe4, 0x7a, 0x36, 0xb6, 0xe4, 0xae, 0xa5, - 0x9b, 0x96, 0xee, 0xb0, 0x53, 0x1c, 0x42, 0xb9, 0xf8, 0xe5, 0xe9, 0xe2, 0xcc, 0x8e, 0x8d, 0xad, - 0x6d, 0x4e, 0x97, 0x66, 0x7a, 0x81, 0x2b, 0xf7, 0x43, 0x51, 0x93, 0xe3, 0x7f, 0x28, 0xea, 0x05, - 0x14, 0x2d, 0xac, 0x68, 0x21, 0x0f, 0x84, 0xe5, 0x20, 0x8a, 0x1e, 0x33, 0xf4, 0x94, 0x95, 0x5b, - 0x92, 0xe6, 0x22, 0x2a, 0x58, 0x61, 0x22, 0x7a, 0x00, 0x97, 0x3a, 0xca, 0x31, 0x8d, 0x89, 0x94, - 0x5d, 0xa7, 0x8e, 0xc6, 0x39, 0xb2, 0x6f, 0x30, 0xa1, 0x8e, 0x72, 0x4c, 0xbf, 0x3a, 0xc5, 0x6e, - 0xd1, 0x4f, 0x49, 0xdc, 0x82, 0xbc, 0xa6, 0xdb, 0x8e, 0x6e, 0xa8, 0x0e, 0xcf, 0x3d, 0xcb, 0xf2, - 0xb6, 0xe6, 0x5c, 0x2a, 0x4b, 0x30, 0x7b, 0x1f, 0x66, 0x79, 0xb0, 0x7c, 0xe0, 0x15, 0x21, 0xcf, - 0xdf, 0xca, 0x6e, 0x78, 0x6f, 0x05, 0x51, 0x05, 0x0a, 0x2d, 0xc5, 0xc1, 0x47, 0xca, 0x89, 0xec, - 0x1e, 0xc5, 0xca, 0xd2, 0x94, 0x8c, 0x6f, 0x9d, 0x9d, 0x2e, 0xe6, 0x9e, 0xb2, 0x5b, 0x03, 0x27, - 0xb2, 0x72, 0xad, 0xc0, 0x0d, 0x0d, 0xdd, 0x81, 0x82, 0x62, 0x9f, 0x18, 0x2a, 0x55, 0x0f, 0x36, - 0xec, 0x9e, 0x4d, 0x21, 0x45, 0x5a, 0xca, 0x53, 0x72, 0xc5, 0xa5, 0xa2, 0xeb, 0x30, 0xc3, 0x23, - 0xc9, 0xd9, 0x67, 0x6c, 0x0a, 0xb4, 0xa9, 0xfc, 0xbb, 0x09, 0xf4, 0x4b, 0x36, 0xe8, 0x31, 0x2c, - 0xf0, 0xdc, 0xf2, 0x47, 0x8a, 0xa5, 0xc9, 0x54, 0xeb, 0xfe, 0xf8, 0x2c, 0x52, 0xb1, 0x57, 0x58, - 0x2e, 0x79, 0x52, 0x80, 0xa8, 0xda, 0xcf, 0x84, 0xb0, 0x05, 0xb3, 0x6a, 0x5b, 0xf7, 0x10, 0x00, - 0x6b, 0xf9, 0xec, 0x50, 0x3b, 0x5b, 0xa1, 0x65, 0xfd, 0x57, 0xa4, 0x05, 0x35, 0x4c, 0x40, 0x0d, - 0xa0, 0xd9, 0x5e, 0xe4, 0xae, 0xd9, 0xd6, 0xd5, 0x13, 0xea, 0xfc, 0x87, 0x73, 0x64, 0x8f, 0xcc, - 0x54, 0xff, 0x4a, 0xd1, 0x9d, 0x6d, 0xca, 0x29, 0xc1, 0x91, 0xf7, 0x9b, 0xe5, 0xb7, 0x5d, 0x4f, - 0xa5, 0x67, 0x8a, 0xb9, 0xf5, 0x54, 0x3a, 0x5f, 0x2c, 0x88, 0x7f, 0x5b, 0x80, 0x42, 0x5f, 0x5d, - 0x50, 0x0d, 0x2e, 0x69, 0xde, 0x54, 0x91, 0xf9, 0x51, 0x23, 0xdd, 0x34, 0x78, 0xda, 0xef, 0xb9, - 0x2f, 0x4f, 0x17, 0x0b, 0xb4, 0xf4, 0x53, 0xef, 0x96, 0x34, 0xef, 0x73, 0xf8, 0x54, 0xf4, 0x11, - 0xe4, 0x99, 0xfb, 0xe8, 0x7d, 0x99, 0x8d, 0xc6, 0x88, 0x97, 0x67, 0xbf, 0x3c, 0x5d, 0xcc, 0x51, - 0x9f, 0xd1, 0x4d, 0xcb, 0x2b, 0xe5, 0xda, 0xc1, 0x4b, 0xf1, 0x37, 0x04, 0x98, 0x09, 0x1d, 0xe7, - 0x79, 0xdc, 0xf7, 0x06, 0xfd, 0x6a, 0x34, 0xee, 0x1c, 0x16, 0x50, 0x97, 0xe6, 0xe3, 0xdc, 0x8d, - 0x4e, 0x5c, 0x1c, 0x8e, 0x5b, 0xe8, 0x2e, 0x8c, 0x1b, 0xaa, 0xe1, 0xb2, 0x7d, 0x9c, 0xfa, 0xfe, - 0x0f, 0x16, 0x27, 0xc4, 0x3f, 0x48, 0x41, 0x2e, 0x7c, 0x78, 0xa7, 0xde, 0x57, 0xaf, 0xa8, 0x75, - 0x21, 0xc4, 0xb1, 0x3c, 0x22, 0x6d, 0x61, 0xc6, 0xcf, 0xaf, 0xcf, 0xaa, 0xb9, 0x34, 0x22, 0x4e, - 0x20, 0x58, 0x4f, 0x9f, 0x71, 0xe1, 0x7b, 0x49, 0xcf, 0xbe, 0x2e, 0xc3, 0x24, 0x4d, 0xa3, 0xc3, - 0xab, 0x56, 0xea, 0x1f, 0x3d, 0xc4, 0x57, 0x26, 0xf7, 0x25, 0x56, 0x8c, 0xd8, 0xe3, 0xe6, 0x1b, - 0xe5, 0xa9, 0xf3, 0xa7, 0xc1, 0xc5, 0x3f, 0x84, 0xc7, 0xf3, 0x14, 0x4e, 0x5e, 0x2c, 0x4f, 0x21, - 0xfa, 0x25, 0x28, 0xa8, 0x66, 0xbb, 0xcd, 0xd6, 0x3a, 0x66, 0x91, 0x06, 0x33, 0x8f, 0x50, 0x11, - 0xfc, 0xdb, 0x85, 0xcb, 0xde, 0x37, 0x0c, 0x97, 0x25, 0xfe, 0x0d, 0xc3, 0x40, 0xac, 0x67, 0xde, - 0x13, 0xc6, 0x0c, 0x59, 0x5f, 0xd8, 0xe9, 0xf4, 0x9b, 0x84, 0x9d, 0xb2, 0x50, 0x65, 0x3e, 0x72, - 0xfe, 0x54, 0xe0, 0x81, 0x21, 0xcf, 0x4c, 0xf3, 0xa0, 0xe7, 0x85, 0x8b, 0x2e, 0x04, 0xb3, 0x06, - 0xa6, 0xbf, 0x3c, 0x5d, 0x4c, 0x49, 0x5e, 0xda, 0xc0, 0x28, 0xcb, 0x9f, 0xf8, 0xc9, 0x2c, 0xff, - 0x75, 0x98, 0xe9, 0x5a, 0x78, 0x0f, 0x3b, 0xea, 0xbe, 0x6c, 0xf4, 0x3a, 0xfc, 0x54, 0x49, 0xd6, - 0xa5, 0x6d, 0xf5, 0x3a, 0xe8, 0x1e, 0x14, 0xbd, 0x22, 0x1c, 0x63, 0xbb, 0x69, 0xa5, 0x5c, 0x3a, - 0x47, 0xe4, 0xe2, 0xff, 0x16, 0x60, 0x2e, 0xd4, 0x26, 0x3e, 0x27, 0xd6, 0x21, 0xeb, 0x9b, 0x03, - 0xbb, 0x24, 0x5c, 0x30, 0x78, 0x32, 0xc8, 0x8c, 0x64, 0xb8, 0xec, 0x3e, 0x96, 0xe6, 0x94, 0xf7, - 0xc5, 0x26, 0x2e, 0x28, 0xf6, 0x92, 0x2f, 0x67, 0x2d, 0xf0, 0x00, 0x6f, 0x92, 0x24, 0xc7, 0x9a, - 0x24, 0xe2, 0x6f, 0x0b, 0x50, 0xa4, 0x0f, 0x78, 0x82, 0xb1, 0x16, 0x8b, 0x75, 0x72, 0x83, 0x92, - 0x13, 0xe3, 0x9f, 0x1b, 0x09, 0x7d, 0x07, 0x23, 0x19, 0xfe, 0x0e, 0x86, 0xf8, 0x03, 0x01, 0xf2, - 0x5e, 0x0d, 0xd9, 0xd7, 0xe0, 0x46, 0x24, 0xa7, 0x7c, 0xb3, 0x2f, 0xa1, 0xb9, 0x59, 0x38, 0xc6, - 0xfa, 0x40, 0x5d, 0x30, 0x0b, 0x07, 0xfb, 0x82, 0xd7, 0xdf, 0x75, 0x47, 0x0e, 0xa9, 0x62, 0xc5, - 0xcf, 0xb6, 0xf0, 0x06, 0x47, 0x68, 0x24, 0xfa, 0x21, 0x4d, 0xb3, 0x7d, 0xc8, 0x12, 0x9f, 0x8c, - 0x65, 0xb6, 0x10, 0x0f, 0x87, 0x02, 0xbe, 0x1b, 0xa7, 0x35, 0x1b, 0xf4, 0x13, 0x9b, 0xec, 0xb7, - 0x2d, 0x3e, 0x09, 0x28, 0x90, 0x76, 0x3e, 0xd1, 0xd2, 0x58, 0xa6, 0xd4, 0xd5, 0x12, 0x1b, 0x2b, - 0x7f, 0x12, 0xec, 0x89, 0xea, 0x21, 0x41, 0x61, 0x8f, 0x20, 0x79, 0xa8, 0xb4, 0x47, 0x85, 0x81, - 0x85, 0x7a, 0x4e, 0x22, 0xa5, 0xd1, 0x93, 0x50, 0x92, 0x8a, 0xc4, 0x70, 0xc4, 0x30, 0xa8, 0xd2, - 0x50, 0x32, 0x8b, 0x0f, 0xc3, 0x63, 0x7d, 0xe4, 0xe3, 0x83, 0x83, 0xfe, 0xe3, 0xd4, 0x0f, 0x7f, - 0xb0, 0x28, 0x88, 0x9f, 0x00, 0x92, 0xb0, 0x8d, 0x9d, 0x17, 0x3d, 0xd3, 0xf2, 0x13, 0x7e, 0xdc, - 0xee, 0xfb, 0x30, 0xc8, 0x64, 0x39, 0x7b, 0x16, 0x95, 0xd2, 0xe7, 0x12, 0xcc, 0x85, 0xb8, 0x99, - 0xb1, 0x10, 0x3f, 0x84, 0xab, 0x4f, 0x4d, 0xdb, 0xd6, 0xbb, 0x04, 0x7a, 0xd2, 0x59, 0x49, 0x96, - 0x06, 0xcf, 0x3c, 0xa6, 0xbb, 0x74, 0x13, 0xc2, 0x60, 0x66, 0x24, 0x23, 0x79, 0xd7, 0xe2, 0x1f, - 0x0a, 0x70, 0x65, 0x90, 0x93, 0x69, 0x39, 0xea, 0xc4, 0xdf, 0xb4, 0x6a, 0xfa, 0xf9, 0xdd, 0xce, - 0x1f, 0xad, 0x6e, 0x71, 0xe2, 0x62, 0xf2, 0x67, 0xca, 0x1d, 0x85, 0x9a, 0x0f, 0x7e, 0xfa, 0x38, - 0xcf, 0xc9, 0x9b, 0x8c, 0xea, 0x5b, 0x92, 0xd4, 0x78, 0x96, 0xa4, 0x09, 0x85, 0x75, 0x53, 0x37, - 0x88, 0x27, 0xeb, 0xb6, 0x77, 0x15, 0xf2, 0xbb, 0xba, 0xa1, 0x58, 0x27, 0xb2, 0x7b, 0xa8, 0x9b, - 0x0d, 0x94, 0x85, 0xa8, 0xca, 0xb2, 0x12, 0x52, 0x8e, 0x71, 0xf0, 0x4b, 0xf1, 0x47, 0x02, 0x14, - 0x7d, 0xb1, 0xdc, 0x22, 0xbf, 0x0b, 0xa0, 0xb6, 0x7b, 0xb6, 0x83, 0x2d, 0xb7, 0x97, 0x66, 0x58, - 0xf4, 0x76, 0x85, 0x51, 0xeb, 0x6b, 0x52, 0x86, 0x17, 0xa8, 0x6b, 0xe8, 0x46, 0x38, 0x39, 0xc2, - 0x64, 0x19, 0xce, 0x06, 0x52, 0x22, 0x90, 0x6e, 0xb7, 0x1d, 0xd3, 0xf2, 0x30, 0x13, 0xef, 0x76, - 0x37, 0x95, 0x3a, 0x3d, 0xd3, 0x4b, 0xca, 0xad, 0x42, 0x9e, 0x2c, 0xf7, 0x87, 0xd8, 0x6b, 0x52, - 0xea, 0xfc, 0x26, 0x31, 0x0e, 0xb7, 0x49, 0xbf, 0x47, 0x9c, 0x54, 0xd6, 0x1b, 0x5e, 0x0f, 0x8f, - 0xb0, 0x68, 0x3f, 0x13, 0x4c, 0x49, 0x38, 0x5e, 0xd2, 0x46, 0xea, 0x89, 0x7c, 0x0b, 0xd2, 0xee, - 0x27, 0x8a, 0xf9, 0x04, 0xb9, 0xba, 0xcc, 0xbe, 0x61, 0xbc, 0xec, 0x7e, 0xc3, 0x78, 0x79, 0x8d, - 0x17, 0x60, 0x66, 0xfc, 0xfb, 0xff, 0x6d, 0x51, 0x90, 0x3c, 0xa6, 0xfb, 0x0d, 0x32, 0xc2, 0x07, - 0x56, 0x61, 0x94, 0x07, 0x08, 0x7c, 0x43, 0x86, 0x7f, 0xb5, 0x76, 0x75, 0x4d, 0xde, 0xd9, 0xaa, - 0x3c, 0xdf, 0xdc, 0xac, 0x37, 0x9b, 0xd5, 0xb5, 0xa2, 0x80, 0x8a, 0x30, 0x13, 0xfa, 0x02, 0x4d, - 0x82, 0x7d, 0xc7, 0xf6, 0xfe, 0xcf, 0x00, 0xf8, 0x1f, 0xb3, 0x22, 0xb2, 0x36, 0xaa, 0x9f, 0xc9, - 0x2f, 0x57, 0x9f, 0xed, 0x54, 0x1b, 0xc5, 0x09, 0x84, 0x20, 0x5f, 0x5e, 0x6d, 0x56, 0x6a, 0xb2, - 0x54, 0x6d, 0x6c, 0x3f, 0xdf, 0x6a, 0x54, 0xdd, 0xef, 0xdf, 0xde, 0x5f, 0x83, 0x99, 0x60, 0xba, - 0x1b, 0x34, 0x07, 0x85, 0x4a, 0xad, 0x5a, 0xd9, 0x90, 0x5f, 0xd6, 0x57, 0xe5, 0x17, 0x3b, 0xd5, - 0x9d, 0x6a, 0x71, 0x82, 0x56, 0x8d, 0x12, 0x9f, 0xec, 0x3c, 0x7b, 0x56, 0x14, 0x50, 0x01, 0xb2, - 0xec, 0x9a, 0x7e, 0xad, 0xa6, 0x98, 0xb8, 0xbf, 0x09, 0xd9, 0x40, 0x5a, 0x5b, 0xf2, 0xb8, 0xed, - 0x9d, 0x46, 0x4d, 0x6e, 0xd6, 0x37, 0xab, 0x8d, 0xe6, 0xea, 0xe6, 0x36, 0x93, 0x41, 0x69, 0xab, - 0xe5, 0xe7, 0x52, 0xb3, 0x28, 0x78, 0xd7, 0xcd, 0xe7, 0x3b, 0x95, 0x9a, 0xdb, 0x0c, 0x31, 0x95, - 0x4e, 0x16, 0x93, 0xf7, 0xff, 0x9a, 0x00, 0x57, 0x86, 0xa4, 0x7e, 0x41, 0x59, 0x98, 0xde, 0x31, - 0x68, 0x8e, 0xcf, 0xe2, 0x04, 0xca, 0x05, 0xb2, 0xbf, 0x14, 0x05, 0x94, 0x66, 0xf9, 0x37, 0x8a, - 0x09, 0x34, 0x05, 0x89, 0xc6, 0xa3, 0x62, 0x92, 0xd4, 0x34, 0x90, 0x3c, 0xa5, 0x98, 0x42, 0x19, - 0x9e, 0xb6, 0xa1, 0x38, 0x89, 0x66, 0xfc, 0xec, 0x09, 0xc5, 0x29, 0x22, 0xca, 0xcb, 0x42, 0x50, - 0x9c, 0xbe, 0x7f, 0x1d, 0x02, 0x27, 0xbd, 0x11, 0xc0, 0xd4, 0x33, 0xc5, 0xc1, 0xb6, 0x53, 0x9c, - 0x40, 0xd3, 0x90, 0x5c, 0x6d, 0xb7, 0x8b, 0xc2, 0xc3, 0x7f, 0x9d, 0x82, 0xb4, 0xfb, 0x55, 0x16, - 0xf4, 0x0c, 0x26, 0xd9, 0xd6, 0xf3, 0xe2, 0x70, 0xcf, 0x9e, 0x4e, 0xde, 0x85, 0xa5, 0xf3, 0x5c, - 0x7f, 0x71, 0x02, 0xfd, 0x15, 0xc8, 0x06, 0x3c, 0x26, 0x34, 0x74, 0xfb, 0x2c, 0xe4, 0x25, 0x2e, - 0xdc, 0x3e, 0xaf, 0x98, 0x27, 0xff, 0x15, 0x64, 0x3c, 0x0b, 0x8e, 0x6e, 0x8c, 0xb2, 0xef, 0xae, - 0xec, 0xd1, 0x8b, 0x00, 0x99, 0x6b, 0xe2, 0xc4, 0xfb, 0x02, 0xb2, 0x00, 0x0d, 0x1a, 0x5b, 0x14, - 0x15, 0x91, 0x30, 0xd4, 0x9a, 0x2f, 0xdc, 0x1f, 0xab, 0xb4, 0xff, 0x4c, 0xa2, 0x2c, 0x7f, 0xc5, - 0x88, 0x56, 0xd6, 0xc0, 0x7a, 0x14, 0xad, 0xac, 0x88, 0x85, 0x67, 0x02, 0xbd, 0x80, 0x14, 0xb1, - 0x94, 0x28, 0xca, 0x87, 0xec, 0xb3, 0xcc, 0x0b, 0x37, 0x46, 0x96, 0x71, 0x45, 0x96, 0xef, 0xfd, - 0xf0, 0x2f, 0xae, 0x4d, 0xfc, 0xf0, 0xec, 0x9a, 0xf0, 0xa3, 0xb3, 0x6b, 0xc2, 0x9f, 0x9d, 0x5d, - 0x13, 0xfe, 0xfc, 0xec, 0x9a, 0xf0, 0xeb, 0x3f, 0xbe, 0x36, 0xf1, 0xa3, 0x1f, 0x5f, 0x9b, 0xf8, - 0xb3, 0x1f, 0x5f, 0x9b, 0xf8, 0x7c, 0x9a, 0x73, 0xef, 0x4e, 0x51, 0xa3, 0xf2, 0xe8, 0xff, 0x05, - 0x00, 0x00, 0xff, 0xff, 0x2a, 0xe5, 0xf1, 0xd1, 0x53, 0x7e, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_2661ffa68e6772f8) } + +var fileDescriptor_api_2661ffa68e6772f8 = []byte{ + // 8188 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0xbd, 0x6b, 0x6c, 0x23, 0x59, + 0x76, 0x1f, 0xae, 0x22, 0x29, 0x89, 0x3c, 0x14, 0x1f, 0xba, 0xea, 0x07, 0x9b, 0x33, 0xd3, 0xea, + 0xae, 0x7e, 0xf7, 0xce, 0x48, 0xd3, 0xdd, 0x3b, 0x9e, 0xf1, 0xf4, 0x78, 0xd6, 0x12, 0xc5, 0x6e, + 0x52, 0x6a, 0xa9, 0xd5, 0x45, 0xaa, 0xdb, 0x33, 0x5e, 0xff, 0x6b, 0x4b, 0x55, 0x57, 0x54, 0xad, + 0xc8, 0x2a, 0x76, 0x55, 0x51, 0x8f, 0x01, 0xfe, 0x40, 0x1c, 0x1b, 0x89, 0x81, 0x00, 0x8b, 0xfd, + 0xe0, 0x20, 0x6b, 0x38, 0x89, 0xd7, 0x71, 0x1c, 0x07, 0x08, 0xf2, 0x00, 0x12, 0xc4, 0x41, 0x90, + 0xd8, 0x5f, 0x8c, 0x64, 0x11, 0x38, 0xc8, 0xfa, 0x53, 0x8c, 0x00, 0x51, 0x6c, 0x6d, 0xbe, 0x25, + 0x08, 0x82, 0x7c, 0x09, 0x30, 0x1f, 0x82, 0xe0, 0x3e, 0xea, 0x45, 0x16, 0x1f, 0xea, 0xad, 0x41, + 0x06, 0xf0, 0x17, 0x89, 0x75, 0xea, 0x9e, 0x53, 0xf7, 0x9e, 0x7b, 0xef, 0xb9, 0xe7, 0x77, 0xeb, + 0xdc, 0x53, 0x30, 0x6f, 0x99, 0x8a, 0xba, 0xdf, 0xdd, 0x5d, 0x56, 0xba, 0xfa, 0x52, 0xd7, 0x32, + 0x1d, 0x13, 0xcd, 0xab, 0xa6, 0x7a, 0x40, 0xc9, 0x4b, 0xfc, 0x66, 0xf9, 0xfe, 0xc1, 0xe1, 0xf2, + 0xc1, 0xa1, 0x8d, 0xad, 0x43, 0x6c, 0x2d, 0xab, 0xa6, 0xa1, 0xf6, 0x2c, 0x0b, 0x1b, 0xea, 0xc9, + 0x72, 0xdb, 0x54, 0x0f, 0xe8, 0x1f, 0xdd, 0x68, 0x31, 0xf6, 0x32, 0x72, 0x25, 0x6a, 0x8a, 0xa3, + 0x70, 0xda, 0x05, 0x97, 0x86, 0x2d, 0xcb, 0xb4, 0x6c, 0x4e, 0xbd, 0xe4, 0x52, 0x3b, 0xd8, 0x51, + 0x02, 0xa5, 0xdf, 0xb2, 0x1d, 0xd3, 0x52, 0x5a, 0x78, 0x19, 0x1b, 0x2d, 0xdd, 0xc0, 0xa4, 0xc0, + 0xa1, 0xaa, 0xf2, 0x9b, 0x6f, 0x47, 0xde, 0x7c, 0xc4, 0xef, 0x96, 0x7a, 0x8e, 0xde, 0x5e, 0xde, + 0x6f, 0xab, 0xcb, 0x8e, 0xde, 0xc1, 0xb6, 0xa3, 0x74, 0xba, 0xfc, 0xce, 0x7d, 0x7a, 0xc7, 0xb1, + 0x14, 0x55, 0x37, 0x5a, 0xee, 0xff, 0xee, 0xee, 0xb2, 0x85, 0x55, 0xd3, 0xd2, 0xb0, 0x26, 0xdb, + 0x5d, 0xc5, 0x70, 0xab, 0xdb, 0x32, 0x5b, 0x26, 0xfd, 0xb9, 0x4c, 0x7e, 0x71, 0xea, 0xd5, 0x96, + 0x69, 0xb6, 0xda, 0x78, 0x99, 0x5e, 0xed, 0xf6, 0xf6, 0x96, 0xb5, 0x9e, 0xa5, 0x38, 0xba, 0xc9, + 0xb9, 0xc4, 0x7f, 0x2e, 0x40, 0x4e, 0xc2, 0xaf, 0x7b, 0xd8, 0x76, 0x6a, 0x58, 0xd1, 0xb0, 0x85, + 0xae, 0x40, 0xf2, 0x00, 0x9f, 0x94, 0x92, 0xd7, 0x84, 0xbb, 0x73, 0xab, 0xb3, 0x5f, 0x9e, 0x2e, + 0x26, 0x37, 0xf0, 0x89, 0x44, 0x68, 0xe8, 0x1a, 0xcc, 0x62, 0x43, 0x93, 0xc9, 0xed, 0x54, 0xf8, + 0xf6, 0x0c, 0x36, 0xb4, 0x0d, 0x7c, 0x82, 0xbe, 0x0d, 0x69, 0x9b, 0x48, 0x33, 0x54, 0x5c, 0x9a, + 0xbe, 0x26, 0xdc, 0x9d, 0x5e, 0xfd, 0xf9, 0x2f, 0x4f, 0x17, 0x3f, 0x69, 0xe9, 0xce, 0x7e, 0x6f, + 0x77, 0x49, 0x35, 0x3b, 0xcb, 0x5e, 0x3f, 0x69, 0xbb, 0xfe, 0xef, 0xe5, 0xee, 0x41, 0x6b, 0xb9, + 0x5f, 0x47, 0x4b, 0xcd, 0x63, 0xa3, 0x81, 0x5f, 0x4b, 0x9e, 0xc4, 0xf5, 0x54, 0x5a, 0x28, 0x26, + 0xd6, 0x53, 0xe9, 0x44, 0x31, 0x29, 0xfe, 0x4e, 0x12, 0xf2, 0x12, 0xb6, 0xbb, 0xa6, 0x61, 0x63, + 0x5e, 0xf3, 0xf7, 0x21, 0xe9, 0x1c, 0x1b, 0xb4, 0xe6, 0xd9, 0x87, 0x57, 0x97, 0x06, 0x46, 0xc4, + 0x52, 0xd3, 0x52, 0x0c, 0x5b, 0x51, 0x49, 0xf3, 0x25, 0x52, 0x14, 0x7d, 0x04, 0x59, 0x0b, 0xdb, + 0xbd, 0x0e, 0xa6, 0x8a, 0xa4, 0x8d, 0xca, 0x3e, 0xbc, 0x1c, 0xc1, 0xd9, 0xe8, 0x2a, 0x86, 0x04, + 0xac, 0x2c, 0xf9, 0x8d, 0xae, 0x40, 0xda, 0xe8, 0x75, 0x88, 0x2a, 0x6c, 0xda, 0xd0, 0xa4, 0x34, + 0x6b, 0xf4, 0x3a, 0x1b, 0xf8, 0xc4, 0x46, 0xbf, 0x00, 0x97, 0x34, 0xdc, 0xb5, 0xb0, 0xaa, 0x38, + 0x58, 0x93, 0x2d, 0xc5, 0x68, 0x61, 0x59, 0x37, 0xf6, 0x4c, 0xbb, 0x34, 0x73, 0x2d, 0x79, 0x37, + 0xfb, 0xf0, 0xed, 0x08, 0xf9, 0x12, 0x29, 0x55, 0x37, 0xf6, 0xcc, 0xd5, 0xd4, 0x8f, 0x4e, 0x17, + 0xa7, 0xa4, 0x0b, 0xbe, 0x04, 0xef, 0x96, 0x8d, 0x1a, 0x90, 0xe3, 0xd5, 0xb5, 0xb0, 0x62, 0x9b, + 0x46, 0x69, 0xf6, 0x9a, 0x70, 0x37, 0xff, 0x70, 0x29, 0x4a, 0x60, 0x48, 0x35, 0xe4, 0xb2, 0xd7, + 0xc1, 0x12, 0xe5, 0x92, 0xe6, 0xac, 0xc0, 0x15, 0x7a, 0x0b, 0x32, 0xa4, 0x25, 0xbb, 0x27, 0x0e, + 0xb6, 0x4b, 0x69, 0xda, 0x14, 0xd2, 0xb4, 0x55, 0x72, 0x2d, 0x7e, 0x0a, 0x73, 0x41, 0x56, 0x84, + 0x20, 0x2f, 0x55, 0x1b, 0x3b, 0x9b, 0x55, 0x79, 0x67, 0x6b, 0x63, 0xeb, 0xf9, 0xab, 0xad, 0xe2, + 0x14, 0xba, 0x00, 0x45, 0x4e, 0xdb, 0xa8, 0x7e, 0x26, 0x3f, 0xab, 0x6f, 0xd6, 0x9b, 0x45, 0xa1, + 0x9c, 0xfa, 0xb5, 0xdf, 0xb9, 0x3a, 0x25, 0x6e, 0x03, 0x3c, 0xc5, 0x0e, 0x1f, 0x60, 0x68, 0x15, + 0x66, 0xf6, 0x69, 0x7d, 0x4a, 0x02, 0xd5, 0xf4, 0xb5, 0xc8, 0x8a, 0x07, 0x06, 0xe3, 0x6a, 0x9a, + 0x68, 0xe3, 0xc7, 0xa7, 0x8b, 0x82, 0xc4, 0x39, 0xc5, 0x3f, 0x14, 0x20, 0x4b, 0x45, 0xb2, 0xf6, + 0xa1, 0x4a, 0x9f, 0xcc, 0xeb, 0x63, 0x95, 0x31, 0x28, 0x14, 0x2d, 0xc1, 0xf4, 0xa1, 0xd2, 0xee, + 0xe1, 0x52, 0x82, 0xca, 0x28, 0x45, 0xc8, 0x78, 0x49, 0xee, 0x4b, 0xac, 0x18, 0x7a, 0x0c, 0x73, + 0xba, 0xe1, 0x60, 0xc3, 0x91, 0x19, 0x5b, 0x72, 0x0c, 0x5b, 0x96, 0x95, 0xa6, 0x17, 0xe2, 0x3f, + 0x13, 0x00, 0xb6, 0x7b, 0x71, 0x2a, 0x05, 0x7d, 0x73, 0xc2, 0xfa, 0xf3, 0xd1, 0xc5, 0x5b, 0x71, + 0x09, 0x66, 0x74, 0xa3, 0xad, 0x1b, 0xac, 0xfe, 0x69, 0x89, 0x5f, 0xa1, 0x0b, 0x30, 0xbd, 0xdb, + 0xd6, 0x0d, 0x8d, 0xce, 0x87, 0xb4, 0xc4, 0x2e, 0x44, 0x09, 0xb2, 0xb4, 0xd6, 0x31, 0xea, 0x5d, + 0xfc, 0x37, 0x09, 0xb8, 0x58, 0x31, 0x0d, 0x4d, 0x27, 0x53, 0x52, 0x69, 0x7f, 0x2d, 0xb4, 0xb2, + 0x0e, 0x81, 0xc9, 0x27, 0xe3, 0xe3, 0xee, 0x84, 0x7d, 0x8c, 0x7c, 0xae, 0xea, 0x71, 0x97, 0xd2, + 0xa2, 0x35, 0x89, 0xbe, 0x09, 0x97, 0x95, 0x76, 0xdb, 0x3c, 0x92, 0xf5, 0x3d, 0x59, 0x33, 0xb1, + 0x2d, 0x1b, 0xa6, 0x23, 0xe3, 0x63, 0xdd, 0x76, 0xa8, 0x29, 0x49, 0x4b, 0x0b, 0xf4, 0x76, 0x7d, + 0x6f, 0xcd, 0xc4, 0xf6, 0x96, 0xe9, 0x54, 0xc9, 0x2d, 0x32, 0x4f, 0x49, 0x65, 0xd8, 0x3c, 0x9d, + 0x21, 0xe6, 0x57, 0x4a, 0xe3, 0xe3, 0x2e, 0x9b, 0xa7, 0xbf, 0x04, 0x97, 0xfa, 0xf5, 0x18, 0x67, + 0x3f, 0xfd, 0xb1, 0x00, 0xf9, 0xba, 0xa1, 0x3b, 0x5f, 0x8b, 0x0e, 0xf2, 0x94, 0x9a, 0x0c, 0x2a, + 0xf5, 0x3e, 0x14, 0xf7, 0x14, 0xbd, 0xfd, 0xdc, 0x68, 0x9a, 0x9d, 0x5d, 0xdb, 0x31, 0x0d, 0x6c, + 0x73, 0xad, 0x0f, 0xd0, 0xc5, 0x97, 0x50, 0xf0, 0x5a, 0x13, 0xa7, 0x9a, 0x1c, 0x28, 0xd6, 0x0d, + 0xd5, 0xc2, 0x1d, 0x6c, 0xc4, 0xaa, 0xa7, 0xb7, 0x21, 0xa3, 0xbb, 0x72, 0xa9, 0xae, 0x92, 0x92, + 0x4f, 0x10, 0x7b, 0x30, 0x1f, 0x78, 0x6a, 0x9c, 0x66, 0x91, 0x2c, 0x0d, 0xf8, 0x48, 0xf6, 0xfb, + 0x88, 0x2c, 0x0d, 0xf8, 0x88, 0x99, 0xb1, 0x06, 0xe4, 0xd6, 0x70, 0x1b, 0x3b, 0x38, 0x4e, 0xeb, + 0xbe, 0x03, 0x79, 0x57, 0x68, 0x9c, 0x1d, 0xf3, 0x37, 0x04, 0x40, 0x5c, 0x2e, 0x59, 0x4d, 0xe3, + 0xec, 0x9b, 0x45, 0xe2, 0x42, 0x38, 0x3d, 0xcb, 0x60, 0xbe, 0x00, 0x1b, 0x93, 0xc0, 0x48, 0xd4, + 0x1d, 0xf0, 0xad, 0x6c, 0x2a, 0x68, 0x65, 0xb9, 0x1b, 0x73, 0x04, 0x0b, 0xa1, 0x8a, 0xc5, 0xdb, + 0x7d, 0x29, 0x5a, 0xa7, 0xc4, 0xb5, 0x64, 0xd0, 0x57, 0xa3, 0x44, 0xf1, 0x07, 0x02, 0xcc, 0x57, + 0xda, 0x58, 0xb1, 0x62, 0xd7, 0xc8, 0xb7, 0x20, 0xad, 0x61, 0x45, 0xa3, 0x4d, 0x66, 0x13, 0xfb, + 0x9d, 0x80, 0x14, 0xe2, 0xd1, 0x2e, 0xed, 0xb7, 0xd5, 0xa5, 0xa6, 0xeb, 0xeb, 0xf2, 0xd9, 0xed, + 0x31, 0x89, 0x9f, 0x01, 0x0a, 0xd6, 0x2c, 0xce, 0x81, 0xf0, 0xb7, 0x05, 0x40, 0x12, 0x3e, 0xc4, + 0x96, 0x13, 0x7b, 0xb3, 0xd7, 0x20, 0xeb, 0x28, 0x56, 0x0b, 0x3b, 0x32, 0xf1, 0xe2, 0xcf, 0xd3, + 0x72, 0x60, 0x7c, 0x84, 0x2c, 0x7e, 0x0e, 0x0b, 0xa1, 0xfa, 0xc5, 0xd9, 0xf8, 0xff, 0x29, 0x40, + 0xb6, 0xa1, 0x2a, 0x46, 0x9c, 0xad, 0xfe, 0x14, 0xb2, 0xb6, 0xaa, 0x18, 0xf2, 0x9e, 0x69, 0x75, + 0x14, 0x87, 0x0e, 0xf1, 0x7c, 0xa8, 0xd5, 0x9e, 0x07, 0xad, 0x2a, 0xc6, 0x13, 0x5a, 0x48, 0x02, + 0xdb, 0xfb, 0x8d, 0x5e, 0x40, 0xf6, 0x00, 0x9f, 0xc8, 0x1c, 0x8d, 0xd1, 0xf5, 0x2f, 0xff, 0xf0, + 0xfd, 0x00, 0xff, 0xc1, 0xe1, 0x92, 0x0b, 0xe2, 0x96, 0x02, 0x20, 0x6e, 0x89, 0x70, 0x2c, 0x35, + 0x1c, 0x0b, 0x1b, 0x2d, 0x67, 0x5f, 0x82, 0x03, 0x7c, 0xf2, 0x8c, 0xc9, 0x60, 0x13, 0x6b, 0x3d, + 0x95, 0x4e, 0x16, 0x53, 0xe2, 0xff, 0x16, 0x60, 0x8e, 0x35, 0x39, 0xce, 0x89, 0xf5, 0x01, 0xa4, + 0x2c, 0xf3, 0x88, 0x4d, 0xac, 0xec, 0xc3, 0xb7, 0x22, 0x44, 0x6c, 0xe0, 0x93, 0xe0, 0xca, 0x45, + 0x8b, 0xa3, 0x55, 0xe0, 0x7e, 0xa0, 0x4c, 0xb9, 0x93, 0x93, 0x72, 0x03, 0xe3, 0x92, 0x88, 0x8c, + 0x3b, 0x50, 0xd8, 0x55, 0x1c, 0x75, 0x5f, 0xb6, 0x78, 0x25, 0xc9, 0x2a, 0x97, 0xbc, 0x3b, 0x27, + 0xe5, 0x29, 0xd9, 0xad, 0xba, 0x4d, 0x5a, 0xce, 0x46, 0xba, 0x8d, 0xff, 0x82, 0xf5, 0xf9, 0xff, + 0x11, 0xf8, 0x1c, 0x72, 0x5b, 0xfe, 0x17, 0xad, 0xeb, 0x7f, 0x23, 0x01, 0x97, 0x2b, 0xfb, 0x58, + 0x3d, 0xa8, 0x98, 0x86, 0xad, 0xdb, 0x0e, 0xd1, 0x5d, 0x9c, 0xfd, 0xff, 0x16, 0x64, 0x8e, 0x74, + 0x67, 0x5f, 0xd6, 0xf4, 0xbd, 0x3d, 0x6a, 0xe7, 0xd2, 0x52, 0x9a, 0x10, 0xd6, 0xf4, 0xbd, 0x3d, + 0xf4, 0x08, 0x52, 0x1d, 0x53, 0x63, 0xee, 0x72, 0xfe, 0xe1, 0x62, 0x84, 0x78, 0x5a, 0x35, 0xbb, + 0xd7, 0xd9, 0x34, 0x35, 0x2c, 0xd1, 0xc2, 0xe8, 0x2a, 0x80, 0x4a, 0xa8, 0x5d, 0x53, 0x37, 0x1c, + 0xbe, 0x4e, 0x06, 0x28, 0xa8, 0x06, 0x19, 0x07, 0x5b, 0x1d, 0xdd, 0x50, 0x1c, 0x5c, 0x9a, 0xa6, + 0xca, 0xbb, 0x19, 0x59, 0xf1, 0x6e, 0x5b, 0x57, 0x95, 0x35, 0x6c, 0xab, 0x96, 0xde, 0x75, 0x4c, + 0x8b, 0x6b, 0xd1, 0x67, 0x16, 0xbf, 0x97, 0x82, 0xd2, 0xa0, 0x6e, 0xe2, 0x1c, 0x21, 0xdb, 0x30, + 0x43, 0xf0, 0x75, 0xdb, 0xe1, 0x63, 0xe4, 0xe1, 0x30, 0x15, 0x44, 0xd4, 0x80, 0xe2, 0xf4, 0xb6, + 0xc3, 0xab, 0xcd, 0xe5, 0x94, 0xff, 0x95, 0x00, 0x33, 0xec, 0x06, 0x7a, 0x00, 0x69, 0xbe, 0xa1, + 0xa0, 0xd1, 0x3a, 0x26, 0x57, 0x2f, 0x9d, 0x9d, 0x2e, 0xce, 0xb2, 0x3d, 0x82, 0xb5, 0x2f, 0xfd, + 0x9f, 0xd2, 0x2c, 0x2d, 0x57, 0xd7, 0x48, 0x6f, 0xd9, 0x8e, 0x62, 0x39, 0x74, 0xdb, 0x26, 0xc1, + 0x70, 0x03, 0x25, 0x6c, 0xe0, 0x13, 0xb4, 0x0e, 0x33, 0xb6, 0xa3, 0x38, 0x3d, 0x9b, 0xf7, 0xd7, + 0xb9, 0x2a, 0xdb, 0xa0, 0x9c, 0x12, 0x97, 0x40, 0x1c, 0x1d, 0x0d, 0x3b, 0x8a, 0xde, 0xa6, 0x1d, + 0x98, 0x91, 0xf8, 0x95, 0xf8, 0x9b, 0x02, 0xcc, 0xb0, 0xa2, 0xe8, 0x32, 0x2c, 0x48, 0x2b, 0x5b, + 0x4f, 0xab, 0x72, 0x7d, 0x6b, 0xad, 0xda, 0xac, 0x4a, 0x9b, 0xf5, 0xad, 0x95, 0x66, 0xb5, 0x38, + 0x85, 0x2e, 0x01, 0x72, 0x6f, 0x54, 0x9e, 0x6f, 0x35, 0xea, 0x8d, 0x66, 0x75, 0xab, 0x59, 0x14, + 0xe8, 0xde, 0x02, 0xa5, 0x07, 0xa8, 0x09, 0x74, 0x13, 0xae, 0xf5, 0x53, 0xe5, 0x46, 0x73, 0xa5, + 0xd9, 0x90, 0xab, 0x8d, 0x66, 0x7d, 0x73, 0xa5, 0x59, 0x5d, 0x2b, 0x26, 0x47, 0x94, 0x22, 0x0f, + 0x91, 0xa4, 0x6a, 0xa5, 0x59, 0x4c, 0x89, 0x0e, 0x5c, 0x94, 0xb0, 0x6a, 0x76, 0xba, 0x3d, 0x07, + 0x93, 0x5a, 0xda, 0x71, 0xce, 0x94, 0xcb, 0x30, 0xab, 0x59, 0x27, 0xb2, 0xd5, 0x33, 0xf8, 0x3c, + 0x99, 0xd1, 0xac, 0x13, 0xa9, 0x67, 0x88, 0xff, 0x44, 0x80, 0x4b, 0xfd, 0x8f, 0x8d, 0x73, 0x10, + 0xbe, 0x80, 0xac, 0xa2, 0x69, 0x58, 0x93, 0x35, 0xdc, 0x76, 0x14, 0xee, 0x8c, 0xdc, 0x0f, 0x48, + 0xe2, 0x9b, 0x6d, 0x4b, 0xde, 0x66, 0xdb, 0xe6, 0xcb, 0x4a, 0x85, 0x56, 0x64, 0x8d, 0x70, 0xb8, + 0xe6, 0x87, 0x0a, 0xa1, 0x14, 0xf1, 0xbf, 0xa7, 0x20, 0x57, 0x35, 0xb4, 0xe6, 0x71, 0xac, 0x6b, + 0xc9, 0x25, 0x98, 0x51, 0xcd, 0x4e, 0x47, 0x77, 0x5c, 0x05, 0xb1, 0x2b, 0xf4, 0xb3, 0x01, 0x27, + 0x32, 0x39, 0x81, 0x2b, 0xe5, 0xbb, 0x8f, 0xe8, 0x3b, 0x70, 0x99, 0x58, 0x4d, 0xcb, 0x50, 0xda, + 0x32, 0x93, 0x26, 0x3b, 0x96, 0xde, 0x6a, 0x61, 0x8b, 0x6f, 0xf0, 0xdd, 0x8d, 0xa8, 0x67, 0x9d, + 0x73, 0x54, 0x28, 0x43, 0x93, 0x95, 0x97, 0x2e, 0xea, 0x51, 0x64, 0xf4, 0x09, 0x00, 0x59, 0x8a, + 0xe8, 0xa6, 0xa1, 0xcd, 0xed, 0xd1, 0xb0, 0x5d, 0x43, 0xd7, 0x04, 0x11, 0x06, 0x72, 0x6d, 0xa3, + 0x65, 0x82, 0x18, 0x5e, 0xf7, 0x74, 0x0b, 0xcb, 0x0f, 0xba, 0x2a, 0x85, 0xf2, 0xe9, 0xd5, 0xfc, + 0xd9, 0xe9, 0x22, 0x48, 0x8c, 0xfc, 0x60, 0xbb, 0x42, 0x10, 0x04, 0xfb, 0xdd, 0x55, 0xd1, 0x2b, + 0xb8, 0x17, 0xd8, 0x91, 0x20, 0x2b, 0x2f, 0x6f, 0x96, 0xe2, 0xc8, 0xfb, 0x7a, 0x6b, 0x1f, 0x5b, + 0xb2, 0xb7, 0x71, 0x4c, 0x77, 0xf0, 0xd2, 0xd2, 0x4d, 0x9f, 0xa1, 0xa2, 0x18, 0xac, 0xf6, 0x2b, + 0x4e, 0x8d, 0x16, 0xf6, 0x74, 0x46, 0x94, 0xdf, 0x35, 0x75, 0xdb, 0x34, 0x4a, 0x19, 0xa6, 0x7c, + 0x76, 0x85, 0xee, 0x41, 0xd1, 0x39, 0x36, 0xe4, 0x7d, 0xac, 0x58, 0xce, 0x2e, 0x56, 0x1c, 0xb2, + 0x4a, 0x03, 0x2d, 0x51, 0x70, 0x8e, 0x8d, 0x5a, 0x80, 0x8c, 0x5e, 0x40, 0x51, 0x37, 0xe4, 0xbd, + 0xb6, 0xde, 0xda, 0x77, 0xe4, 0x23, 0x4b, 0x77, 0xb0, 0x5d, 0x9a, 0xa7, 0x0a, 0x89, 0x1a, 0xb7, + 0x0d, 0xbe, 0x93, 0xab, 0xbd, 0x22, 0x25, 0xb9, 0x6a, 0xf2, 0xba, 0xf1, 0x84, 0xf2, 0x53, 0xa2, + 0xbd, 0x9e, 0x4a, 0xcf, 0x16, 0xd3, 0xe2, 0x7f, 0x16, 0x20, 0xef, 0x0e, 0xb7, 0x38, 0x67, 0xc6, + 0x5d, 0x28, 0x9a, 0x06, 0x96, 0xbb, 0xfb, 0x8a, 0x8d, 0xb9, 0x1e, 0xf9, 0x82, 0x93, 0x37, 0x0d, + 0xbc, 0x4d, 0xc8, 0x4c, 0x5d, 0x68, 0x1b, 0xe6, 0x6d, 0x47, 0x69, 0xe9, 0x46, 0x2b, 0xa0, 0xde, + 0xe9, 0xc9, 0xdd, 0xfa, 0x22, 0xe7, 0xf6, 0xe8, 0x21, 0x2f, 0xe5, 0x4f, 0x04, 0x98, 0x5f, 0xd1, + 0x3a, 0xba, 0xd1, 0xe8, 0xb6, 0xf5, 0x58, 0x77, 0x0b, 0x6e, 0x42, 0xc6, 0x26, 0x32, 0x7d, 0x83, + 0xef, 0x63, 0xbf, 0x34, 0xbd, 0x43, 0x2c, 0xff, 0x33, 0x28, 0xe0, 0xe3, 0xae, 0xce, 0x5e, 0x06, + 0x30, 0xc8, 0x92, 0x9a, 0xbc, 0x6d, 0x79, 0x9f, 0x97, 0xdc, 0xe2, 0x6d, 0xfa, 0x0c, 0x50, 0xb0, + 0x49, 0x71, 0x62, 0x97, 0xcf, 0x60, 0x81, 0x8a, 0xde, 0x31, 0xec, 0x98, 0xf5, 0x25, 0xfe, 0x22, + 0x5c, 0x08, 0x8b, 0x8e, 0xb3, 0xde, 0xaf, 0x78, 0x2f, 0x6f, 0x62, 0x2b, 0x56, 0xb8, 0xe9, 0xe9, + 0x9a, 0x0b, 0x8e, 0xb3, 0xce, 0xbf, 0x2a, 0xc0, 0x15, 0x2a, 0x9b, 0xbe, 0x2f, 0xd9, 0xc3, 0xd6, + 0x33, 0xac, 0xd8, 0xb1, 0x62, 0xe5, 0x1b, 0x30, 0xc3, 0x30, 0x2f, 0x1d, 0x9f, 0xd3, 0xab, 0x59, + 0xe2, 0xb9, 0x34, 0x1c, 0xd3, 0x22, 0x9e, 0x0b, 0xbf, 0x25, 0x2a, 0x50, 0x8e, 0xaa, 0x45, 0xcc, + 0xdb, 0x01, 0xf3, 0xdc, 0x69, 0x24, 0x43, 0xb9, 0xb2, 0x4f, 0x7c, 0x26, 0x54, 0x85, 0xac, 0x4a, + 0x7f, 0xc9, 0xce, 0x49, 0x17, 0x53, 0xf9, 0xf9, 0x51, 0xfe, 0x26, 0x63, 0x6b, 0x9e, 0x74, 0x31, + 0x71, 0x5a, 0xdd, 0xdf, 0x44, 0x51, 0x81, 0x46, 0x8e, 0xf4, 0x58, 0xe9, 0x3c, 0xa2, 0x65, 0x5d, + 0xd7, 0x8f, 0xeb, 0xe0, 0x9f, 0x26, 0xb9, 0x12, 0xd8, 0x33, 0x78, 0xf1, 0x58, 0x7d, 0x94, 0xcf, + 0x43, 0xaf, 0xab, 0x82, 0x0d, 0x4f, 0x9c, 0xa3, 0xe1, 0x81, 0x3d, 0x73, 0x9f, 0x8a, 0x3e, 0x83, + 0xc0, 0xae, 0xb8, 0xcc, 0xda, 0xe4, 0xa2, 0x9f, 0xf3, 0xa8, 0x63, 0xde, 0x97, 0xc2, 0xe8, 0x36, + 0xaa, 0x40, 0x1a, 0x1f, 0x77, 0x65, 0x0d, 0xdb, 0x2a, 0x37, 0x5c, 0xe2, 0xb0, 0xf7, 0x6a, 0x03, + 0x78, 0x60, 0x16, 0x1f, 0x77, 0x09, 0x11, 0xed, 0x90, 0xd5, 0xcb, 0x75, 0x15, 0x68, 0xb5, 0xed, + 0xf1, 0xf0, 0xc2, 0x1f, 0x29, 0x5c, 0x5c, 0xc1, 0xf3, 0x12, 0x98, 0x08, 0xf1, 0x87, 0x02, 0xbc, + 0x15, 0xd9, 0x6b, 0x71, 0x2e, 0x64, 0x9f, 0x40, 0x8a, 0x36, 0x3e, 0x71, 0xce, 0xc6, 0x53, 0x2e, + 0xf1, 0x77, 0xdd, 0x39, 0x2e, 0xe1, 0xb6, 0x49, 0x14, 0xfb, 0x15, 0xec, 0x87, 0xcd, 0xba, 0x1d, + 0x9e, 0x38, 0x77, 0x87, 0xbb, 0xac, 0x9e, 0x11, 0xe8, 0xab, 0x66, 0x9c, 0x46, 0xe0, 0xfb, 0x02, + 0x2c, 0x78, 0x3e, 0x4d, 0xcc, 0xee, 0xed, 0x07, 0x90, 0x34, 0xcc, 0xa3, 0xf3, 0x6c, 0x06, 0x92, + 0xf2, 0x64, 0x49, 0x0a, 0xd7, 0x28, 0xce, 0xf6, 0xfe, 0xeb, 0x04, 0x64, 0x9e, 0x56, 0xe2, 0x6c, + 0xe5, 0x27, 0x7c, 0xa3, 0x99, 0x4d, 0xec, 0xa8, 0xa1, 0xe8, 0x3d, 0x6f, 0xe9, 0x69, 0x65, 0x03, + 0x9f, 0xb8, 0x43, 0x91, 0x70, 0xa1, 0x15, 0xc8, 0x38, 0xfb, 0x16, 0xb6, 0xf7, 0xcd, 0xb6, 0x76, + 0x1e, 0x1f, 0xc4, 0xe7, 0x2a, 0x63, 0x98, 0xa6, 0x72, 0xdd, 0xe0, 0x05, 0x21, 0x22, 0x78, 0x81, + 0x3c, 0xc6, 0x73, 0xe3, 0x12, 0xe7, 0x79, 0x4c, 0xc0, 0x7f, 0x9b, 0x2e, 0xce, 0x88, 0x2f, 0x00, + 0x48, 0x73, 0xe2, 0xec, 0x92, 0xbf, 0x92, 0x84, 0xfc, 0x76, 0xcf, 0xde, 0x8f, 0x79, 0xf4, 0x55, + 0x00, 0xba, 0x3d, 0x9b, 0xe2, 0x83, 0x63, 0x83, 0xb7, 0x79, 0x4c, 0x5c, 0x84, 0xdb, 0x68, 0xc6, + 0xd7, 0x3c, 0x36, 0x50, 0x8d, 0x0b, 0xc1, 0xb2, 0x1f, 0x5c, 0x71, 0x63, 0x14, 0x92, 0x6c, 0x1e, + 0x1b, 0x9b, 0xd8, 0x83, 0x90, 0x4c, 0x12, 0x26, 0x92, 0x3e, 0x81, 0x59, 0x72, 0x21, 0x3b, 0xe6, + 0x79, 0xba, 0x79, 0x86, 0xf0, 0x34, 0x4d, 0xf4, 0x18, 0x32, 0x8c, 0x9b, 0x2c, 0x4d, 0x33, 0x74, + 0x69, 0x8a, 0x6a, 0x0b, 0x57, 0x23, 0x5d, 0x94, 0xd2, 0x94, 0x95, 0x2c, 0x44, 0x17, 0x60, 0x7a, + 0xcf, 0xb4, 0x54, 0x4c, 0x23, 0x26, 0xd2, 0x12, 0xbb, 0x60, 0xfd, 0xb9, 0x9e, 0x4a, 0xa7, 0x8b, + 0x99, 0xf5, 0x54, 0x3a, 0x53, 0x04, 0xf1, 0x37, 0x05, 0x28, 0x78, 0x1d, 0x11, 0xa7, 0xb5, 0xae, + 0x84, 0xb4, 0x78, 0xfe, 0xae, 0x20, 0x0a, 0x14, 0xff, 0x2d, 0x75, 0x57, 0x54, 0xf3, 0x90, 0xf6, + 0x4c, 0x9c, 0x23, 0xe5, 0x31, 0x0b, 0x9d, 0x49, 0x9c, 0xb7, 0x77, 0x69, 0x14, 0xcd, 0x03, 0xb8, + 0xa0, 0x77, 0x88, 0x1d, 0xd7, 0x9d, 0xf6, 0x09, 0xc7, 0x54, 0x0e, 0x76, 0xdf, 0xcf, 0x2e, 0xf8, + 0xf7, 0x2a, 0xee, 0x2d, 0xf1, 0xef, 0xd2, 0xdd, 0x69, 0xbf, 0x25, 0x71, 0xaa, 0xba, 0x0e, 0x39, + 0x8b, 0x89, 0x26, 0x3e, 0xc7, 0x39, 0xb5, 0x3d, 0xe7, 0xb1, 0x12, 0x85, 0xff, 0x76, 0x02, 0x0a, + 0x2f, 0x7a, 0xd8, 0x3a, 0xf9, 0x3a, 0xa9, 0xfb, 0x36, 0x14, 0x8e, 0x14, 0xdd, 0x91, 0xf7, 0x4c, + 0x4b, 0xee, 0x75, 0x35, 0xc5, 0x71, 0xe3, 0x37, 0x72, 0x84, 0xfc, 0xc4, 0xb4, 0x76, 0x28, 0x11, + 0x61, 0x40, 0x07, 0x86, 0x79, 0x64, 0xc8, 0x84, 0x4c, 0x51, 0xec, 0xb1, 0xc1, 0xb7, 0x8c, 0x57, + 0x3f, 0xfc, 0x4f, 0xa7, 0x8b, 0x8f, 0x26, 0x8a, 0xca, 0xa2, 0x11, 0x68, 0xbd, 0x9e, 0xae, 0x2d, + 0xed, 0xec, 0xd4, 0xd7, 0xa4, 0x22, 0x15, 0xf9, 0x8a, 0x49, 0x6c, 0x1e, 0x1b, 0xb6, 0xf8, 0xf7, + 0x12, 0x50, 0xf4, 0x75, 0x14, 0x67, 0x47, 0x56, 0x21, 0xfb, 0xba, 0x87, 0x2d, 0xfd, 0x0d, 0xba, + 0x11, 0x38, 0x23, 0x31, 0x3b, 0x9f, 0xc3, 0x5c, 0x48, 0x03, 0xc9, 0x9f, 0x4e, 0x03, 0xd9, 0x23, + 0xbf, 0xf1, 0xe8, 0x3e, 0xcc, 0x3b, 0xc7, 0x86, 0xcc, 0xe2, 0xf1, 0x58, 0x0c, 0x87, 0x1b, 0x76, + 0x50, 0x70, 0x88, 0x3e, 0x08, 0x9d, 0xc6, 0x6f, 0xd8, 0xe2, 0x1f, 0x0a, 0x80, 0xa8, 0xa2, 0xea, + 0x6c, 0x4f, 0xff, 0xeb, 0x32, 0x9e, 0xee, 0x42, 0x91, 0x46, 0x38, 0xca, 0xfa, 0x9e, 0xdc, 0xd1, + 0x6d, 0x5b, 0x37, 0x5a, 0x7c, 0x40, 0xe5, 0x29, 0xbd, 0xbe, 0xb7, 0xc9, 0xa8, 0xe2, 0xff, 0x0f, + 0x0b, 0xa1, 0x06, 0xc4, 0xd9, 0xd9, 0xd7, 0x61, 0x6e, 0xcf, 0xec, 0x19, 0x9a, 0xcc, 0xde, 0x78, + 0xf0, 0xed, 0xc0, 0x2c, 0xa5, 0xb1, 0xe7, 0x89, 0xff, 0x2d, 0x01, 0x17, 0x24, 0x6c, 0x9b, 0xed, + 0x43, 0x1c, 0xbf, 0x0a, 0x6b, 0xc0, 0xdf, 0xb5, 0xc8, 0x6f, 0xa4, 0xc9, 0x0c, 0x63, 0x66, 0xcb, + 0x5c, 0x78, 0x4f, 0xfd, 0xe6, 0xe8, 0x11, 0x3b, 0xb8, 0x8b, 0xce, 0xf7, 0xe4, 0x52, 0xa1, 0x3d, + 0x39, 0x13, 0x0a, 0x7a, 0xcb, 0x30, 0x89, 0x4d, 0xb3, 0xf1, 0x6b, 0xa3, 0xd7, 0x71, 0x91, 0xca, + 0xd2, 0xa8, 0x4a, 0xd6, 0x19, 0x4b, 0x03, 0xbf, 0xde, 0xea, 0x75, 0xa8, 0xef, 0xbc, 0x7a, 0x89, + 0xd4, 0xf7, 0xec, 0x74, 0x31, 0x1f, 0xba, 0x67, 0x4b, 0x79, 0xdd, 0xbb, 0x26, 0xd2, 0xc5, 0x6f, + 0xc3, 0xc5, 0x3e, 0x65, 0xc7, 0xe9, 0xf1, 0xfc, 0xcb, 0x24, 0x5c, 0x09, 0x8b, 0x8f, 0x1b, 0x7f, + 0x7c, 0xdd, 0x3b, 0xb4, 0x06, 0xb9, 0x8e, 0x6e, 0xbc, 0xd9, 0xd6, 0xe2, 0x5c, 0x47, 0x37, 0xfc, + 0x6d, 0xdc, 0x88, 0xa1, 0x31, 0xf3, 0x95, 0x0e, 0x0d, 0x05, 0xca, 0x51, 0x7d, 0x17, 0xe7, 0xf8, + 0xf8, 0x35, 0x01, 0xe6, 0xe2, 0xde, 0x33, 0x7b, 0xb3, 0x78, 0x33, 0xb1, 0x09, 0xb9, 0xaf, 0x60, + 0x93, 0xed, 0xb7, 0x05, 0x40, 0x4d, 0xab, 0x67, 0x10, 0x50, 0xfb, 0xcc, 0x6c, 0xc5, 0xd9, 0xcc, + 0x0b, 0x30, 0xad, 0x1b, 0x1a, 0x3e, 0xa6, 0xcd, 0x4c, 0x49, 0xec, 0x22, 0xf4, 0xea, 0x30, 0x39, + 0xd1, 0xab, 0x43, 0xf1, 0x73, 0x58, 0x08, 0x55, 0x31, 0xce, 0xf6, 0xff, 0xfd, 0x04, 0x2c, 0xf0, + 0x86, 0xc4, 0xbe, 0xbd, 0xf8, 0x4d, 0x98, 0x6e, 0x13, 0x99, 0x23, 0xfa, 0x99, 0x3e, 0xd3, 0xed, + 0x67, 0x5a, 0x18, 0xfd, 0x1c, 0x40, 0xd7, 0xc2, 0x87, 0x32, 0x63, 0x4d, 0x4e, 0xc4, 0x9a, 0x21, + 0x1c, 0x94, 0x80, 0x7e, 0x01, 0x0a, 0x64, 0x3e, 0x77, 0x2d, 0xb3, 0x6b, 0xda, 0xc4, 0x65, 0xb1, + 0x27, 0x43, 0x39, 0xf3, 0x67, 0xa7, 0x8b, 0xb9, 0x4d, 0xdd, 0xd8, 0xe6, 0x8c, 0xcd, 0x86, 0x44, + 0x0c, 0x83, 0x77, 0x69, 0x8b, 0xff, 0x41, 0x80, 0x0b, 0x5f, 0xd9, 0x56, 0xec, 0xff, 0x0b, 0x5d, + 0x89, 0x2f, 0xa1, 0x48, 0x7f, 0xd4, 0x8d, 0x3d, 0x33, 0xce, 0x4d, 0xf1, 0xef, 0x09, 0x30, 0x1f, + 0x10, 0x1c, 0xa7, 0x7f, 0xf2, 0x46, 0x7a, 0x12, 0x7f, 0x91, 0x78, 0x2c, 0xc1, 0x41, 0x1e, 0xe7, + 0x14, 0xfa, 0xfd, 0x04, 0x5c, 0xaa, 0xb0, 0x57, 0xc8, 0x6e, 0x4c, 0x45, 0x9c, 0x23, 0xa3, 0x04, + 0xb3, 0x87, 0xd8, 0xb2, 0x75, 0x93, 0xad, 0x9e, 0x39, 0xc9, 0xbd, 0x44, 0x65, 0x48, 0xdb, 0x86, + 0xd2, 0xb5, 0xf7, 0x4d, 0xf7, 0xdd, 0x99, 0x77, 0xed, 0xc5, 0x7f, 0x4c, 0xbf, 0x79, 0xfc, 0xc7, + 0xcc, 0xe8, 0xf8, 0x8f, 0xd9, 0x9f, 0x22, 0xfe, 0x83, 0xbf, 0xa8, 0xfa, 0x77, 0x02, 0x5c, 0x1e, + 0xd0, 0x5c, 0x9c, 0xa3, 0xe5, 0xbb, 0x90, 0x55, 0xb9, 0x60, 0x62, 0x6f, 0xd9, 0x5b, 0xb8, 0x3a, + 0x29, 0xf6, 0x86, 0xb0, 0xe3, 0xec, 0x74, 0x11, 0xdc, 0xaa, 0xd6, 0xd7, 0xb8, 0x72, 0xc8, 0x6f, + 0x4d, 0xfc, 0x95, 0x1c, 0x14, 0xaa, 0xc7, 0x6c, 0x07, 0xba, 0xc1, 0x56, 0x79, 0xf4, 0x04, 0xd2, + 0x5d, 0xcb, 0x3c, 0xd4, 0xdd, 0x66, 0xe4, 0x43, 0x2f, 0xff, 0xdd, 0x66, 0xf4, 0x71, 0x6d, 0x73, + 0x0e, 0xc9, 0xe3, 0x45, 0x4d, 0xc8, 0x3c, 0x33, 0x55, 0xa5, 0xfd, 0x44, 0x6f, 0xbb, 0x23, 0xff, + 0xfd, 0xf1, 0x82, 0x96, 0x3c, 0x9e, 0x6d, 0xc5, 0xd9, 0x77, 0x3b, 0xc1, 0x23, 0xa2, 0x3a, 0xa4, + 0x6b, 0x8e, 0xd3, 0x25, 0x37, 0xb9, 0xed, 0xb8, 0x33, 0x81, 0x50, 0xc2, 0xe2, 0xc6, 0x8a, 0xba, + 0xec, 0xa8, 0x09, 0xf3, 0x4f, 0xe9, 0x09, 0xa7, 0x4a, 0xdb, 0xec, 0x69, 0x15, 0xd3, 0xd8, 0xd3, + 0x5b, 0xdc, 0xee, 0xde, 0x9e, 0x40, 0xe6, 0xd3, 0x4a, 0x43, 0x1a, 0x14, 0x80, 0x56, 0x20, 0xdd, + 0x78, 0xc4, 0x85, 0x31, 0xb7, 0xec, 0xd6, 0x04, 0xc2, 0x1a, 0x8f, 0x24, 0x8f, 0x0d, 0xad, 0x43, + 0x76, 0xe5, 0x8b, 0x9e, 0x85, 0xb9, 0x94, 0x99, 0xa1, 0x91, 0x07, 0xfd, 0x52, 0x28, 0x97, 0x14, + 0x64, 0x46, 0x0d, 0xc8, 0xbf, 0x32, 0xad, 0x83, 0xb6, 0xa9, 0xb8, 0x2d, 0x9c, 0xa5, 0xe2, 0xbe, + 0x31, 0x81, 0x38, 0x97, 0x51, 0xea, 0x13, 0x81, 0xbe, 0x0d, 0x05, 0xd2, 0x19, 0x4d, 0x65, 0xb7, + 0xed, 0x56, 0x32, 0x4d, 0xa5, 0xbe, 0x3b, 0x81, 0x54, 0x8f, 0xd3, 0x7d, 0x05, 0xd2, 0x27, 0xaa, + 0x2c, 0x41, 0x2e, 0x34, 0x08, 0x10, 0x82, 0x54, 0x97, 0xf4, 0xb7, 0x40, 0x63, 0x83, 0xe8, 0x6f, + 0xf4, 0x1e, 0xcc, 0x1a, 0xa6, 0x86, 0xdd, 0x19, 0x92, 0x5b, 0xbd, 0x70, 0x76, 0xba, 0x38, 0xb3, + 0x65, 0x6a, 0xcc, 0x21, 0xe1, 0xbf, 0xa4, 0x19, 0x52, 0xa8, 0xae, 0x95, 0xaf, 0x41, 0x8a, 0xf4, + 0x3b, 0x31, 0x4c, 0xbb, 0x8a, 0x8d, 0x77, 0x2c, 0x9d, 0x4b, 0x73, 0x2f, 0xcb, 0xff, 0x38, 0x01, + 0x89, 0xc6, 0x23, 0xe2, 0x72, 0xef, 0xf6, 0xd4, 0x03, 0xec, 0xf0, 0xfb, 0xfc, 0x8a, 0xba, 0xe2, + 0x16, 0xde, 0xd3, 0x99, 0x67, 0x94, 0x91, 0xf8, 0x15, 0x7a, 0x07, 0x40, 0x51, 0x55, 0x6c, 0xdb, + 0xb2, 0x7b, 0xf2, 0x2d, 0x23, 0x65, 0x18, 0x65, 0x03, 0x9f, 0x10, 0x36, 0x1b, 0xab, 0x16, 0x76, + 0xdc, 0xc0, 0x26, 0x76, 0x45, 0xd8, 0x1c, 0xdc, 0xe9, 0xca, 0x8e, 0x79, 0x80, 0x0d, 0x3a, 0x4e, + 0x32, 0xc4, 0xd4, 0x74, 0xba, 0x4d, 0x42, 0x20, 0x56, 0x12, 0x1b, 0x9a, 0x6f, 0xd2, 0x32, 0x92, + 0x77, 0x4d, 0x44, 0x5a, 0xb8, 0xa5, 0xf3, 0x23, 0x5c, 0x19, 0x89, 0x5f, 0x11, 0x2d, 0x29, 0x3d, + 0x67, 0x9f, 0xf6, 0x44, 0x46, 0xa2, 0xbf, 0xd1, 0x6d, 0x28, 0xb0, 0x58, 0x48, 0x19, 0x1b, 0xaa, + 0x4c, 0x8d, 0x6b, 0x86, 0xde, 0xce, 0x31, 0x72, 0xd5, 0x50, 0x89, 0x29, 0x45, 0x8f, 0x80, 0x13, + 0xe4, 0x83, 0x8e, 0x4d, 0x74, 0x0a, 0xa4, 0xd4, 0x6a, 0xe1, 0xec, 0x74, 0x31, 0xdb, 0xa0, 0x37, + 0x36, 0x36, 0x1b, 0xf5, 0x35, 0x29, 0xcb, 0x4a, 0x6d, 0x74, 0xec, 0xba, 0x56, 0xfe, 0x75, 0x01, + 0x92, 0x4f, 0x2b, 0x8d, 0x73, 0xab, 0xcc, 0xad, 0x68, 0x32, 0x50, 0xd1, 0x3b, 0x50, 0xd8, 0xd5, + 0xdb, 0x6d, 0xdd, 0x68, 0x11, 0x2f, 0xe8, 0xbb, 0x58, 0x75, 0x15, 0x96, 0xe7, 0xe4, 0x6d, 0x46, + 0x45, 0xd7, 0x20, 0xab, 0x5a, 0x58, 0xc3, 0x86, 0xa3, 0x2b, 0x6d, 0x9b, 0x6b, 0x2e, 0x48, 0x2a, + 0xff, 0xb2, 0x00, 0xd3, 0x74, 0x06, 0xa0, 0xb7, 0x21, 0xa3, 0x9a, 0x86, 0xa3, 0xe8, 0x06, 0x37, + 0x65, 0x19, 0xc9, 0x27, 0x0c, 0xad, 0xde, 0x75, 0x98, 0x53, 0x54, 0xd5, 0xec, 0x19, 0x8e, 0x6c, + 0x28, 0x1d, 0xcc, 0xab, 0x99, 0xe5, 0xb4, 0x2d, 0xa5, 0x83, 0xd1, 0x22, 0xb8, 0x97, 0xde, 0x81, + 0xc6, 0x8c, 0x04, 0x9c, 0xb4, 0x81, 0x4f, 0xca, 0x7f, 0x24, 0x40, 0xda, 0x9d, 0x33, 0xa4, 0x1a, + 0x2d, 0x6c, 0x60, 0x4b, 0x71, 0x4c, 0xaf, 0x1a, 0x1e, 0xa1, 0x7f, 0xa9, 0xcc, 0xf8, 0x4b, 0xe5, + 0x05, 0x98, 0x76, 0xc8, 0xb4, 0xe0, 0x35, 0x60, 0x17, 0x74, 0x3b, 0xba, 0xad, 0xb4, 0xd8, 0x6e, + 0x5c, 0x46, 0x62, 0x17, 0xa4, 0x31, 0x3c, 0xa4, 0x96, 0x69, 0x84, 0x5f, 0x91, 0x9a, 0xb2, 0xc0, + 0xcf, 0x5d, 0xdc, 0xd2, 0x0d, 0x3a, 0x96, 0x92, 0x12, 0x50, 0xd2, 0x2a, 0xa1, 0xa0, 0xb7, 0x20, + 0xc3, 0x0a, 0x60, 0x43, 0xa3, 0x03, 0x2a, 0x29, 0xa5, 0x29, 0xa1, 0x6a, 0x68, 0x65, 0x0c, 0x19, + 0x6f, 0x72, 0x92, 0x6e, 0xeb, 0xd9, 0x9e, 0x22, 0xe9, 0x6f, 0xf4, 0x3e, 0x5c, 0x78, 0xdd, 0x53, + 0xda, 0xfa, 0x1e, 0xdd, 0x68, 0x23, 0xc5, 0x98, 0xce, 0x58, 0x4b, 0x90, 0x77, 0x8f, 0x4a, 0xa0, + 0xaa, 0x73, 0xe7, 0x72, 0xd2, 0x9f, 0xcb, 0xe2, 0xef, 0x09, 0x30, 0xcf, 0xa2, 0x7a, 0x58, 0x30, + 0x6a, 0x7c, 0x7e, 0xc8, 0xc7, 0x90, 0xd1, 0x14, 0x47, 0x61, 0x47, 0x34, 0x13, 0x23, 0x8f, 0x68, + 0x7a, 0x47, 0x09, 0x14, 0x47, 0xa1, 0xc7, 0x34, 0x11, 0xa4, 0xc8, 0x6f, 0x76, 0x9a, 0x55, 0xa2, + 0xbf, 0xc5, 0xcf, 0x00, 0x05, 0x2b, 0x1a, 0xa7, 0x47, 0x76, 0x0f, 0x2e, 0x12, 0x5d, 0x57, 0x0d, + 0xd5, 0x3a, 0xe9, 0x3a, 0xba, 0x69, 0x3c, 0xa7, 0x7f, 0x6d, 0x54, 0x0c, 0xbc, 0x97, 0xa2, 0xaf, + 0xa3, 0xc4, 0x3f, 0x98, 0x81, 0x5c, 0xf5, 0xb8, 0x6b, 0x5a, 0xb1, 0xee, 0x62, 0xad, 0xc2, 0x2c, + 0x07, 0xfa, 0x23, 0xde, 0x0b, 0xf7, 0x19, 0x73, 0xf7, 0x95, 0x2b, 0x67, 0x44, 0xab, 0x00, 0x2c, + 0x60, 0x94, 0x06, 0x05, 0x25, 0xcf, 0xf1, 0xa6, 0x8c, 0xb2, 0x11, 0x2a, 0xda, 0x82, 0x6c, 0xe7, + 0x50, 0x55, 0xe5, 0x3d, 0xbd, 0xed, 0xf0, 0xb8, 0xbb, 0xe8, 0x10, 0xf1, 0xcd, 0x97, 0x95, 0xca, + 0x13, 0x5a, 0x88, 0x85, 0xc0, 0xf9, 0xd7, 0x12, 0x10, 0x09, 0xec, 0x37, 0x7a, 0x17, 0xf8, 0x91, + 0x1a, 0xd9, 0x76, 0x4f, 0xc9, 0xad, 0xe6, 0xce, 0x4e, 0x17, 0x33, 0x12, 0xa5, 0x36, 0x1a, 0x4d, + 0x29, 0xc3, 0x0a, 0x34, 0x6c, 0x07, 0xdd, 0x80, 0x9c, 0xd9, 0xd1, 0x1d, 0xd9, 0x75, 0x92, 0xb8, + 0x47, 0x39, 0x47, 0x88, 0xae, 0x13, 0x85, 0x9a, 0x70, 0x07, 0x1b, 0x74, 0xb4, 0x93, 0x76, 0xca, + 0xbb, 0x6c, 0xf3, 0xd1, 0x61, 0x33, 0x5a, 0x36, 0xbb, 0x8e, 0xde, 0xd1, 0xbf, 0xa0, 0x6f, 0xa6, + 0xf9, 0x4b, 0xa3, 0x1b, 0xac, 0x38, 0x69, 0xdf, 0x2a, 0xdd, 0x95, 0xe4, 0x65, 0x9f, 0x07, 0x8a, + 0xa2, 0xbf, 0x2a, 0xc0, 0x25, 0xae, 0x48, 0x79, 0x97, 0xc6, 0xb8, 0x2b, 0x6d, 0xdd, 0x39, 0x91, + 0x0f, 0x0e, 0x4b, 0x69, 0xea, 0xb7, 0xfe, 0x6c, 0x64, 0x87, 0x04, 0xc6, 0xc1, 0x92, 0xdb, 0x2d, + 0x27, 0xcf, 0x38, 0xf3, 0xc6, 0x61, 0xd5, 0x70, 0xac, 0x93, 0xd5, 0xcb, 0x67, 0xa7, 0x8b, 0x0b, + 0x83, 0x77, 0x5f, 0x4a, 0x0b, 0xf6, 0x20, 0x0b, 0xaa, 0x01, 0x60, 0x6f, 0x1c, 0xd2, 0x15, 0x23, + 0xda, 0xff, 0x88, 0x1c, 0xb0, 0x52, 0x80, 0x17, 0xdd, 0x85, 0x22, 0x3f, 0xd9, 0xb2, 0xa7, 0xb7, + 0xb1, 0x6c, 0xeb, 0x5f, 0x60, 0xba, 0xb6, 0x24, 0xa5, 0x3c, 0xa3, 0x13, 0x11, 0x0d, 0xfd, 0x0b, + 0x5c, 0xfe, 0x2e, 0x94, 0x86, 0xd5, 0x3e, 0x38, 0x05, 0x32, 0xec, 0x8d, 0xec, 0x47, 0xe1, 0xed, + 0x98, 0x09, 0x86, 0x2a, 0xdf, 0x92, 0xf9, 0x38, 0xf1, 0x91, 0x20, 0xfe, 0x83, 0x04, 0xe4, 0x56, + 0x7b, 0xed, 0x83, 0xe7, 0xdd, 0x46, 0xaf, 0xd3, 0x51, 0xac, 0x13, 0x62, 0x06, 0x99, 0xa1, 0x20, + 0x15, 0x14, 0x98, 0x19, 0xa4, 0x96, 0x40, 0xff, 0x02, 0x93, 0xc5, 0x29, 0x78, 0x2a, 0x9b, 0xc5, + 0xf0, 0xd3, 0x36, 0x04, 0x8e, 0x5a, 0x9b, 0x47, 0x36, 0xfa, 0x08, 0x4a, 0x81, 0x82, 0x74, 0xef, + 0x44, 0xc6, 0x86, 0x63, 0xe9, 0x98, 0xed, 0xff, 0x25, 0xa5, 0x40, 0xbc, 0x4c, 0x9d, 0xdc, 0xae, + 0xb2, 0xbb, 0xa8, 0x09, 0x73, 0xa4, 0xe0, 0x89, 0x4c, 0x97, 0x10, 0x77, 0x7f, 0xf6, 0x41, 0x44, + 0xb3, 0x42, 0xf5, 0x5e, 0xa2, 0xfa, 0xa9, 0x50, 0x1e, 0xfa, 0x53, 0xca, 0x62, 0x9f, 0x52, 0xfe, + 0x14, 0x8a, 0xfd, 0x05, 0x82, 0xba, 0x4c, 0x31, 0x5d, 0x5e, 0x08, 0xea, 0x32, 0x19, 0xd0, 0xd3, + 0x7a, 0x2a, 0x9d, 0x2a, 0x4e, 0x8b, 0x7f, 0x9e, 0x84, 0xbc, 0x3b, 0xcc, 0xe2, 0x04, 0x3a, 0xab, + 0x30, 0x4d, 0x06, 0x85, 0x1b, 0xe3, 0x71, 0x7b, 0xc4, 0xe8, 0xe6, 0x51, 0xe3, 0x64, 0xb0, 0xb8, + 0x20, 0x99, 0xb2, 0xc6, 0x61, 0x70, 0xca, 0xbf, 0x9c, 0x80, 0x14, 0xc5, 0x16, 0x0f, 0x20, 0x45, + 0x17, 0x0a, 0x61, 0x92, 0x85, 0x82, 0x16, 0xf5, 0x96, 0xb3, 0x44, 0xc0, 0x35, 0x25, 0x3e, 0xdf, + 0xbe, 0xf2, 0xc1, 0x83, 0x87, 0xd4, 0xd8, 0xcc, 0x49, 0xfc, 0x0a, 0xad, 0xd2, 0xb0, 0x23, 0xd3, + 0x72, 0xb0, 0xc6, 0x7d, 0xfa, 0x6b, 0xe3, 0xfa, 0xd7, 0x5d, 0x94, 0x5c, 0x3e, 0x74, 0x05, 0x92, + 0xc4, 0x8a, 0xcd, 0xb2, 0x20, 0x85, 0xb3, 0xd3, 0xc5, 0x24, 0xb1, 0x5f, 0x84, 0x86, 0x96, 0x21, + 0x1b, 0x36, 0x19, 0xc4, 0x83, 0xa3, 0x86, 0x31, 0x30, 0xdd, 0xa1, 0xed, 0x4d, 0x2d, 0x86, 0x67, + 0x79, 0x1f, 0xff, 0x8f, 0x14, 0xe4, 0xea, 0x9d, 0xb8, 0x97, 0x94, 0x95, 0x70, 0x0f, 0x47, 0x01, + 0xa1, 0xd0, 0x43, 0x23, 0x3a, 0x38, 0xb4, 0x82, 0x27, 0xcf, 0xb7, 0x82, 0xd7, 0x89, 0xa7, 0xcc, + 0xd3, 0x2c, 0x24, 0x87, 0x60, 0x9e, 0xf0, 0xf3, 0xa9, 0x9f, 0x22, 0x11, 0x1e, 0xff, 0x1c, 0x05, + 0x0d, 0x34, 0xf9, 0x94, 0x3a, 0xe4, 0x6c, 0x94, 0xcd, 0x4c, 0x3e, 0xca, 0x66, 0xb1, 0xa1, 0xd1, + 0x45, 0x2d, 0x6c, 0x51, 0x67, 0xdf, 0xdc, 0xa2, 0x96, 0x1d, 0x3e, 0x58, 0x3f, 0x86, 0xa4, 0xa6, + 0xbb, 0x9d, 0x33, 0xf9, 0x52, 0x4d, 0x98, 0xc6, 0x8c, 0xda, 0x54, 0x70, 0xd4, 0xb2, 0x51, 0x52, + 0xae, 0x03, 0xf8, 0xba, 0x41, 0xd7, 0x60, 0xc6, 0x6c, 0x6b, 0xee, 0x41, 0x92, 0xdc, 0x6a, 0xe6, + 0xec, 0x74, 0x71, 0xfa, 0x79, 0x5b, 0xab, 0xaf, 0x49, 0xd3, 0x66, 0x5b, 0xab, 0x6b, 0x34, 0xc7, + 0x05, 0x3e, 0x92, 0xbd, 0x28, 0xb3, 0x39, 0x69, 0xd6, 0xc0, 0x47, 0x6b, 0xd8, 0x56, 0xf9, 0x80, + 0xfb, 0x2d, 0x01, 0xf2, 0xae, 0xee, 0xe3, 0x35, 0x2a, 0x69, 0xbd, 0xc3, 0x27, 0x59, 0xf2, 0x7c, + 0x93, 0xcc, 0xe5, 0xe3, 0xc7, 0x6b, 0x7f, 0x55, 0xe0, 0x71, 0xc3, 0x0d, 0x55, 0x71, 0x88, 0x53, + 0x11, 0xe3, 0xc4, 0xb8, 0x07, 0x45, 0x4b, 0x31, 0x34, 0xb3, 0xa3, 0x7f, 0x81, 0xd9, 0x46, 0xa8, + 0xcd, 0xdf, 0x5a, 0x16, 0x3c, 0x3a, 0xdd, 0xf5, 0xb3, 0xc5, 0x3f, 0x4a, 0xf0, 0x18, 0x63, 0xaf, + 0x1a, 0x71, 0xaa, 0xeb, 0x3b, 0x30, 0xdf, 0x9f, 0x70, 0xc4, 0x9d, 0xad, 0xef, 0x45, 0xc8, 0x8b, + 0xaa, 0x08, 0x8b, 0x15, 0x74, 0x03, 0xd7, 0xfb, 0x92, 0x8f, 0xd8, 0xa8, 0x02, 0xd9, 0x60, 0x1e, + 0x93, 0xe4, 0xc4, 0x79, 0x4c, 0xc0, 0xf2, 0xb2, 0x97, 0x94, 0x7f, 0x1e, 0xa6, 0xe9, 0xed, 0x37, + 0x30, 0xd1, 0xbc, 0x37, 0xbf, 0x2f, 0x40, 0x7e, 0x53, 0x6f, 0x59, 0x4a, 0xac, 0x87, 0xce, 0xd1, + 0xc7, 0x61, 0xf4, 0x96, 0x7d, 0x58, 0x8e, 0x7a, 0x31, 0xc4, 0x4a, 0xb8, 0x36, 0x81, 0x33, 0x88, + 0x2f, 0xa1, 0xe0, 0xd5, 0x28, 0x4e, 0x24, 0xf1, 0x67, 0x09, 0xb8, 0x49, 0x3b, 0xea, 0x25, 0xb6, + 0xf4, 0xbd, 0x93, 0x6d, 0xcb, 0x74, 0xb0, 0xea, 0x60, 0xcd, 0x3f, 0xf0, 0x12, 0xab, 0x89, 0xcf, + 0x74, 0xdd, 0x07, 0x9c, 0x2b, 0x34, 0xce, 0xe3, 0x42, 0x1b, 0x50, 0xe0, 0x41, 0x10, 0x4a, 0x5b, + 0x3f, 0xc4, 0xb2, 0xe2, 0x9c, 0x67, 0x21, 0xcf, 0x31, 0xde, 0x15, 0xc2, 0xba, 0xe2, 0x20, 0x0d, + 0x32, 0x5c, 0x98, 0xae, 0xf1, 0x4c, 0x43, 0x4f, 0x7f, 0xba, 0xbd, 0xd3, 0x34, 0x8b, 0xc4, 0xa8, + 0xaf, 0x49, 0x69, 0x26, 0xb9, 0xae, 0x89, 0xff, 0x51, 0x80, 0x5b, 0x63, 0x54, 0x1c, 0xe7, 0x2c, + 0x2d, 0x43, 0xfa, 0x90, 0x3c, 0x48, 0xe7, 0x3a, 0x4e, 0x4b, 0xde, 0x35, 0xda, 0x84, 0xdc, 0x9e, + 0xa2, 0xb7, 0xfd, 0xd9, 0x3b, 0x3c, 0x92, 0x32, 0x3a, 0xa8, 0x77, 0x8e, 0xb1, 0xb3, 0xe9, 0x2a, + 0xfe, 0x46, 0x02, 0xe6, 0x57, 0x34, 0xad, 0xd1, 0xe0, 0xe6, 0x3e, 0xbe, 0x91, 0xe2, 0xe2, 0xe9, + 0x84, 0x8f, 0xa7, 0xd1, 0x7b, 0x80, 0x34, 0xdd, 0x66, 0x19, 0x4d, 0xec, 0x7d, 0x45, 0x33, 0x8f, + 0xfc, 0x08, 0x92, 0x79, 0xf7, 0x4e, 0xc3, 0xbd, 0x81, 0x1a, 0x40, 0x81, 0x9d, 0x6c, 0x3b, 0x8a, + 0xf7, 0x8a, 0xec, 0xd6, 0x44, 0x27, 0xd3, 0x18, 0xe2, 0xf3, 0x2e, 0xa5, 0x0c, 0x91, 0x43, 0x7f, + 0x12, 0x88, 0xa2, 0x93, 0xa6, 0x3b, 0xb2, 0x62, 0xbb, 0xc7, 0x90, 0x58, 0x2e, 0x95, 0x3c, 0xa3, + 0xaf, 0xd8, 0xec, 0x74, 0x11, 0x3b, 0x37, 0xe1, 0xab, 0x26, 0xce, 0x39, 0xfb, 0x77, 0x04, 0xc8, + 0x4b, 0x78, 0xcf, 0xc2, 0x76, 0xac, 0xfb, 0x1f, 0x4f, 0x60, 0xce, 0x62, 0x52, 0xe5, 0x3d, 0xcb, + 0xec, 0x9c, 0x67, 0x5e, 0x65, 0x39, 0xe3, 0x13, 0xcb, 0xec, 0x70, 0x1b, 0xfa, 0x12, 0x0a, 0x5e, + 0x1d, 0xe3, 0x6c, 0xfc, 0xef, 0xd1, 0x53, 0xd7, 0x4c, 0x70, 0xdc, 0xa1, 0x1c, 0xf1, 0x6a, 0x80, + 0xbe, 0x93, 0x0b, 0x56, 0x34, 0x4e, 0x35, 0xfc, 0x57, 0x01, 0xf2, 0x8d, 0xde, 0x2e, 0x4b, 0x99, + 0x15, 0x9f, 0x06, 0xaa, 0x90, 0x69, 0xe3, 0x3d, 0x47, 0x7e, 0xa3, 0x88, 0xff, 0x34, 0x61, 0xa5, + 0xe7, 0x1d, 0x9e, 0x02, 0x58, 0xf4, 0xa4, 0x1e, 0x95, 0x93, 0x3c, 0xa7, 0x9c, 0x0c, 0xe5, 0x25, + 0x64, 0xf1, 0x1f, 0x26, 0xa0, 0xe0, 0x35, 0x33, 0x4e, 0x2b, 0xf9, 0x2a, 0x64, 0x1d, 0x92, 0xe7, + 0xb1, 0x0e, 0xf3, 0x3c, 0x7a, 0x25, 0xda, 0x42, 0x2c, 0xc1, 0x02, 0xf5, 0xd1, 0x64, 0xa5, 0xdb, + 0x6d, 0xeb, 0x2e, 0xb2, 0xa7, 0xf6, 0x27, 0x25, 0xcd, 0xd3, 0x5b, 0x2b, 0xec, 0x0e, 0xc5, 0xf4, + 0x64, 0xcc, 0xed, 0x59, 0x18, 0x7f, 0x81, 0x65, 0x0a, 0x32, 0xcf, 0x13, 0x9d, 0x93, 0x65, 0x8c, + 0x0d, 0xc2, 0xc7, 0xc7, 0xdc, 0x2b, 0x98, 0xa7, 0x3a, 0x8d, 0xfb, 0x84, 0xb1, 0xf8, 0xb7, 0x12, + 0x80, 0x82, 0x92, 0xbf, 0xba, 0xbe, 0x48, 0xc4, 0xd7, 0x17, 0xef, 0x02, 0x62, 0xf1, 0x99, 0xb6, + 0xdc, 0xc5, 0x96, 0x6c, 0x63, 0xd5, 0xe4, 0xe9, 0x9c, 0x04, 0xa9, 0xc8, 0xef, 0x6c, 0x63, 0xab, + 0x41, 0xe9, 0xe8, 0x31, 0x80, 0xef, 0x7c, 0xf2, 0x05, 0x63, 0xa4, 0xef, 0x29, 0x65, 0x3c, 0xaf, + 0x53, 0xfc, 0xf7, 0x65, 0x98, 0xe3, 0x3a, 0xdc, 0x31, 0x74, 0xd3, 0x40, 0x0f, 0x20, 0xd9, 0xe2, + 0x6f, 0x39, 0xb2, 0x91, 0x3b, 0x92, 0x7e, 0xbe, 0xba, 0xda, 0x94, 0x44, 0xca, 0x12, 0x96, 0x6e, + 0xcf, 0x89, 0x70, 0x8c, 0xfc, 0x98, 0xf3, 0x20, 0x4b, 0xb7, 0xe7, 0xa0, 0x06, 0x14, 0x54, 0x3f, + 0x1f, 0x97, 0x4c, 0xd8, 0x93, 0x43, 0xf1, 0x62, 0x64, 0x06, 0xb4, 0xda, 0x94, 0x94, 0x57, 0x43, + 0x37, 0x50, 0x25, 0x98, 0x06, 0x2a, 0x35, 0x10, 0xd0, 0xe6, 0x1f, 0x65, 0x0e, 0xa7, 0xa0, 0xaa, + 0x4d, 0x05, 0xb2, 0x45, 0xa1, 0x8f, 0x61, 0x46, 0xa3, 0x09, 0x87, 0xf8, 0x88, 0x8e, 0x1a, 0x74, + 0xa1, 0xbc, 0x4e, 0xb5, 0x29, 0x89, 0x73, 0xa0, 0x75, 0x98, 0x63, 0xbf, 0x98, 0x9b, 0xc2, 0x41, + 0xf4, 0xad, 0xe1, 0x12, 0x02, 0x0b, 0x41, 0x6d, 0x4a, 0xca, 0x6a, 0x3e, 0x15, 0x7d, 0x13, 0x52, + 0xb6, 0xaa, 0xb8, 0x30, 0xfa, 0xea, 0x90, 0xec, 0x21, 0x3e, 0x33, 0x2d, 0x8d, 0x1e, 0xb3, 0x0c, + 0x94, 0xce, 0xb1, 0xbb, 0xa3, 0x19, 0x55, 0xfd, 0xd0, 0x29, 0x75, 0x52, 0x7d, 0x4c, 0x09, 0xe8, + 0x29, 0x64, 0x15, 0xe2, 0xef, 0xc9, 0xf4, 0x9c, 0x27, 0xdd, 0xc2, 0x8c, 0x8e, 0x23, 0x18, 0x38, + 0x97, 0x5b, 0xa3, 0x47, 0xe1, 0x5d, 0xa2, 0x2f, 0xa8, 0x83, 0xad, 0x16, 0x2e, 0x65, 0x47, 0x0b, + 0x0a, 0x86, 0xb1, 0x79, 0x82, 0x28, 0x91, 0xf8, 0x7d, 0xde, 0x21, 0x6b, 0xda, 0xa8, 0xb9, 0xa1, + 0x6f, 0xae, 0x23, 0x4e, 0x28, 0xd5, 0xa6, 0xa4, 0xb9, 0xfd, 0x00, 0x19, 0x2d, 0x41, 0xa2, 0xa5, + 0x96, 0x72, 0x43, 0x67, 0x88, 0x77, 0x0a, 0xa7, 0x36, 0x25, 0x25, 0x5a, 0x2a, 0xfa, 0x14, 0xd2, + 0xec, 0x48, 0xc5, 0xb1, 0x51, 0xca, 0x0f, 0xb5, 0x13, 0xe1, 0x83, 0x29, 0xb5, 0x29, 0x89, 0x9e, + 0xe2, 0x20, 0xcf, 0xdb, 0x86, 0xbc, 0xc5, 0xe2, 0x00, 0xdd, 0xa8, 0xdd, 0xe2, 0xd0, 0xb7, 0xf9, + 0x51, 0x81, 0xbb, 0x35, 0xea, 0xf9, 0x07, 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, 0x47, 0xf2, 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, 0xb9, 0x33, 0x74, 0x51, 0xbc, 0x3a, 0x74, 0x31, 0xe8, 0x0f, 0xb3, + 0x23, 0x8b, 0x41, 0xdb, 0xa5, 0x11, 0x43, 0x46, 0xc1, 0x92, 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, 0x6b, 0xfe, + 0x09, 0x35, 0x43, 0xd7, 0x87, 0xfa, 0xa4, 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, 0x76, 0x01, 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, 0xc3, 0x78, 0x67, + 0xa8, 0xc3, 0x13, 0x99, 0x70, 0x87, 0x38, 0x3c, 0x56, 0xe8, 0x06, 0xfa, 0x39, 0x98, 0xe5, 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, 0xca, 0x87, 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, 0xcf, 0x2f, 0xd0, 0x0f, 0x3a, 0x6a, 0xee, 0x9e, + 0x2c, 0xd3, 0xb0, 0x67, 0xa8, 0x2c, 0x7e, 0xbe, 0x9a, 0x6b, 0xea, 0xbd, 0xd1, 0x86, 0x2a, 0xea, + 0xd0, 0xb8, 0x67, 0xa8, 0x42, 0x37, 0x69, 0x55, 0xd9, 0x91, 0x3a, 0x3a, 0xbf, 0x97, 0x46, 0xa4, + 0x42, 0xe8, 0x3b, 0xd8, 0x48, 0xab, 0xea, 0x11, 0xfd, 0x29, 0xd4, 0x63, 0x79, 0x3a, 0x4a, 0xcb, + 0xa3, 0xa7, 0x50, 0x38, 0x53, 0x88, 0x37, 0x85, 0x38, 0xd9, 0x5b, 0x33, 0x5d, 0x0f, 0xe3, 0xfd, + 0xd1, 0x6b, 0x66, 0xbf, 0x6b, 0xc1, 0xd6, 0x4c, 0xee, 0x53, 0xfc, 0x65, 0x01, 0xae, 0xb1, 0xba, + 0xd1, 0x1d, 0xbd, 0x13, 0xd9, 0xdb, 0x17, 0x0d, 0x1c, 0xef, 0x78, 0x40, 0x1f, 0xf0, 0xe1, 0xb0, + 0xea, 0x8e, 0xd9, 0xe7, 0xad, 0x4d, 0x49, 0xef, 0x28, 0xa3, 0xca, 0x91, 0x21, 0xd5, 0x61, 0x3b, + 0xd1, 0xa5, 0x87, 0x43, 0x87, 0x54, 0x78, 0xf7, 0x9c, 0x0c, 0x29, 0xce, 0xb3, 0x3a, 0xcb, 0x5f, + 0x20, 0x7b, 0xc7, 0x61, 0x0b, 0xc5, 0xe2, 0x7a, 0x2a, 0x7d, 0xb9, 0x58, 0x5a, 0x4f, 0xa5, 0xaf, + 0x14, 0xcb, 0xeb, 0xa9, 0xf4, 0x5b, 0xc5, 0xb7, 0xc5, 0x7f, 0x54, 0x86, 0x9c, 0x0b, 0x1c, 0x19, + 0xa0, 0x7a, 0x18, 0x04, 0x54, 0x57, 0x87, 0x01, 0x2a, 0x0e, 0x35, 0x39, 0xa2, 0x7a, 0x18, 0x44, + 0x54, 0x57, 0x87, 0x21, 0x2a, 0x9f, 0x87, 0x40, 0xaa, 0xe6, 0x30, 0x48, 0x75, 0x6f, 0x02, 0x48, + 0xe5, 0x89, 0xea, 0xc7, 0x54, 0x6b, 0x83, 0x98, 0xea, 0xe6, 0x68, 0x4c, 0xe5, 0x89, 0x0a, 0x80, + 0xaa, 0xc7, 0x7d, 0xa0, 0xea, 0xfa, 0x08, 0x50, 0xe5, 0xf1, 0xbb, 0xa8, 0x6a, 0x23, 0x12, 0x55, + 0xdd, 0x1e, 0x87, 0xaa, 0x3c, 0x39, 0x21, 0x58, 0xf5, 0x41, 0x08, 0x56, 0x2d, 0x0e, 0x85, 0x55, + 0x1e, 0x37, 0xc3, 0x55, 0x9f, 0xf4, 0xe3, 0xaa, 0xeb, 0x23, 0x70, 0x95, 0xdf, 0x02, 0x0e, 0xac, + 0x6a, 0x51, 0xc0, 0xea, 0xd6, 0x18, 0x60, 0xe5, 0x49, 0x09, 0x22, 0xab, 0x5a, 0x14, 0xb2, 0xba, + 0x35, 0x06, 0x59, 0xf5, 0x49, 0x62, 0xd0, 0x6a, 0x2b, 0x1a, 0x5a, 0xdd, 0x19, 0x0b, 0xad, 0x3c, + 0x69, 0x61, 0x6c, 0xb5, 0x1c, 0xc0, 0x56, 0xef, 0x0c, 0xc1, 0x56, 0x1e, 0x2b, 0x01, 0x57, 0xdf, + 0x1a, 0x00, 0x57, 0xe2, 0x28, 0x70, 0xe5, 0xf1, 0x7a, 0xe8, 0xea, 0xc5, 0x10, 0x74, 0x75, 0x77, + 0x3c, 0xba, 0xf2, 0x84, 0xf5, 0xc1, 0x2b, 0x65, 0x24, 0xbc, 0x7a, 0x6f, 0x42, 0x78, 0xe5, 0x49, + 0x8f, 0xc2, 0x57, 0x1f, 0x85, 0xf1, 0xd5, 0xb5, 0xe1, 0xf8, 0xca, 0x13, 0xc3, 0x01, 0xd6, 0x46, + 0x24, 0xc0, 0xba, 0x3d, 0x0e, 0x60, 0xf9, 0xf3, 0x20, 0x88, 0xb0, 0xb6, 0xa2, 0x11, 0xd6, 0x9d, + 0xb1, 0x08, 0xcb, 0xef, 0xfe, 0x10, 0xc4, 0xda, 0x88, 0x84, 0x58, 0xb7, 0xc7, 0x41, 0x2c, 0xbf, + 0x72, 0x41, 0x8c, 0xf5, 0x6a, 0x28, 0xc6, 0xba, 0x3f, 0x09, 0xc6, 0xf2, 0x84, 0x0e, 0x80, 0xac, + 0xcf, 0x87, 0x83, 0xac, 0x6f, 0x9c, 0x23, 0xb3, 0x63, 0x24, 0xca, 0xfa, 0xd6, 0x00, 0xca, 0x12, + 0x47, 0xa1, 0x2c, 0x7f, 0x3c, 0xbb, 0x30, 0x4b, 0x19, 0x09, 0x8a, 0xde, 0x9b, 0x10, 0x14, 0xf9, + 0x83, 0x2f, 0x02, 0x15, 0x55, 0x23, 0x50, 0xd1, 0xcd, 0xd1, 0xa8, 0xc8, 0x37, 0xe7, 0x3e, 0x2c, + 0xaa, 0x45, 0xc1, 0xa2, 0x5b, 0x63, 0x60, 0x91, 0x6f, 0x85, 0x02, 0xb8, 0xe8, 0x71, 0x1f, 0x2e, + 0xba, 0x3e, 0x36, 0x3e, 0x2a, 0x00, 0x8c, 0x56, 0x07, 0x81, 0xd1, 0x8d, 0x91, 0xc0, 0xc8, 0x93, + 0xe0, 0x23, 0xa3, 0xc7, 0x7d, 0xc8, 0xe8, 0xfa, 0x08, 0x64, 0xe4, 0x57, 0x80, 0x43, 0x23, 0x6d, + 0x34, 0x34, 0x5a, 0x9a, 0x14, 0x1a, 0x79, 0x82, 0x23, 0xb1, 0xd1, 0x56, 0x34, 0x36, 0xba, 0x33, + 0x61, 0xe8, 0xc2, 0x00, 0x38, 0xaa, 0x45, 0x81, 0xa3, 0x5b, 0x63, 0xc0, 0x51, 0x70, 0x0d, 0xf1, + 0xd0, 0x51, 0x2d, 0x0a, 0x1d, 0xdd, 0x1a, 0x83, 0x8e, 0x7c, 0x49, 0x01, 0x78, 0xd4, 0x1c, 0x06, + 0x8f, 0xee, 0x4d, 0x00, 0x8f, 0x7c, 0xe7, 0xa5, 0x0f, 0x1f, 0x7d, 0xda, 0x8f, 0x8f, 0xc4, 0x51, + 0xf8, 0xc8, 0x9f, 0x91, 0x2e, 0x40, 0xda, 0x8a, 0x06, 0x48, 0x77, 0xc6, 0x02, 0xa4, 0xa0, 0x91, + 0x0c, 0x20, 0xa4, 0x8d, 0x48, 0x84, 0x74, 0x7b, 0x1c, 0x42, 0xf2, 0x8d, 0x64, 0x10, 0x22, 0x7d, + 0xda, 0x0f, 0x91, 0xc4, 0x51, 0x10, 0xc9, 0x6f, 0x9c, 0x8b, 0x91, 0x6a, 0x51, 0x18, 0xe9, 0xd6, + 0x18, 0x8c, 0xe4, 0x77, 0x5e, 0x00, 0x24, 0x29, 0x23, 0x41, 0xd2, 0x7b, 0x13, 0x82, 0xa4, 0x3e, + 0xc3, 0x15, 0x46, 0x49, 0xb5, 0x28, 0x94, 0x74, 0x6b, 0x0c, 0x4a, 0x0a, 0x54, 0xd6, 0x87, 0x49, + 0x5b, 0xd1, 0x30, 0xe9, 0xce, 0x58, 0x98, 0xd4, 0x37, 0x9b, 0x5c, 0x9c, 0xb4, 0x11, 0x89, 0x93, + 0x6e, 0x8f, 0xc3, 0x49, 0x7d, 0x0b, 0x1f, 0x77, 0x0e, 0x7e, 0x65, 0x72, 0xa0, 0xf4, 0xd1, 0xf9, + 0x81, 0x92, 0xf7, 0xcc, 0x31, 0x48, 0xe9, 0xd3, 0x7e, 0xa4, 0x24, 0x8e, 0x42, 0x4a, 0xfe, 0xc8, + 0x3a, 0x1f, 0x54, 0x5a, 0x4f, 0xa5, 0xdf, 0x2e, 0xbe, 0x23, 0xfe, 0xb5, 0x59, 0x98, 0xa9, 0x79, + 0x61, 0x36, 0x7e, 0x2b, 0x85, 0x37, 0xc9, 0x40, 0x85, 0xd6, 0xc8, 0x8c, 0xa7, 0x76, 0x73, 0x7c, + 0x52, 0xc1, 0xc1, 0xb4, 0x77, 0x9c, 0xf5, 0x0d, 0x8e, 0x82, 0xa3, 0x0f, 0x20, 0xd7, 0xb3, 0xb1, + 0x25, 0x77, 0x2d, 0xdd, 0xb4, 0x74, 0x87, 0x9d, 0x97, 0x11, 0x56, 0x8b, 0x5f, 0x9e, 0x2e, 0xce, + 0xed, 0xd8, 0xd8, 0xda, 0xe6, 0x74, 0x69, 0xae, 0x17, 0xb8, 0x72, 0x3f, 0xc9, 0x35, 0x3d, 0xf9, + 0x27, 0xb9, 0x5e, 0x40, 0xd1, 0xc2, 0x8a, 0x16, 0xf2, 0x60, 0x58, 0xb6, 0xa7, 0xe8, 0x31, 0x47, + 0xcf, 0xb3, 0xb9, 0x25, 0x69, 0xd6, 0xa7, 0x82, 0x15, 0x26, 0xa2, 0x07, 0x70, 0xb1, 0xa3, 0x1c, + 0xd3, 0xe8, 0x53, 0xd9, 0x75, 0x0a, 0x69, 0x44, 0x29, 0xfb, 0xda, 0x15, 0xea, 0x28, 0xc7, 0xf4, + 0xfb, 0x5e, 0xec, 0x16, 0xfd, 0x68, 0xc7, 0x2d, 0xc8, 0x6b, 0xba, 0xed, 0xe8, 0x86, 0xea, 0xf0, + 0x2c, 0xbf, 0x2c, 0x43, 0x6e, 0xce, 0xa5, 0xb2, 0x54, 0xbe, 0xf7, 0x61, 0x9e, 0x1f, 0x4b, 0x08, + 0xbc, 0xa1, 0xe4, 0x99, 0x72, 0xd9, 0x0d, 0xef, 0xa5, 0x24, 0xaa, 0x40, 0xa1, 0xa5, 0x38, 0xf8, + 0x48, 0x39, 0x91, 0xdd, 0x43, 0x6f, 0x59, 0x9a, 0xfc, 0xf2, 0xad, 0xb3, 0xd3, 0xc5, 0xdc, 0x53, + 0x76, 0x6b, 0xe0, 0xec, 0x5b, 0xae, 0x15, 0xb8, 0xa1, 0xa1, 0x3b, 0x50, 0x50, 0xec, 0x13, 0x43, + 0xa5, 0xea, 0xc1, 0x86, 0xdd, 0xb3, 0x29, 0x24, 0x49, 0x4b, 0x79, 0x4a, 0xae, 0xb8, 0x54, 0x74, + 0x1d, 0xe6, 0x78, 0xcc, 0x3e, 0xfb, 0x60, 0x50, 0x81, 0x36, 0x95, 0x7f, 0xa1, 0x82, 0x7e, 0x33, + 0x08, 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, 0x37, 0xb4, 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, 0xba, 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, 0x0d, 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, 0x17, 0xf8, 0x57, 0xa2, 0x71, 0xeb, 0xb0, 0x78, 0xbe, 0x34, + 0x1f, 0xe7, 0x6e, 0x1c, 0xe8, 0xe2, 0x70, 0xdc, 0x43, 0x77, 0x71, 0xdc, 0x48, 0x11, 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, 0x08, 0x53, 0x08, 0xd6, 0xd3, 0x67, 0x2c, 0x7f, 0x2f, 0xe9, 0xd9, 0xd7, 0x25, 0x98, + 0xa6, 0x09, 0x8b, 0x78, 0xd5, 0x4a, 0xfd, 0xa3, 0x87, 0xf8, 0xda, 0xe4, 0xbe, 0xc4, 0x8a, 0x11, + 0x7b, 0xdc, 0x7c, 0xa3, 0x8c, 0x80, 0xfe, 0x34, 0x38, 0xff, 0x27, 0x07, 0x79, 0x46, 0xc8, 0xe9, + 0xf3, 0x65, 0x84, 0x44, 0xbf, 0x04, 0x05, 0xd5, 0x6c, 0xb7, 0xd9, 0x5a, 0xc9, 0x2c, 0xd2, 0x60, + 0x8e, 0x17, 0x2a, 0x82, 0x7f, 0x25, 0x72, 0xc9, 0xfb, 0x5a, 0xe4, 0x92, 0xc4, 0xbf, 0x16, 0x19, + 0x88, 0xaa, 0xcd, 0x7b, 0xc2, 0x98, 0x21, 0xeb, 0x0b, 0xf0, 0x9d, 0x7d, 0x93, 0x00, 0x5f, 0x16, + 0x14, 0xce, 0x47, 0xce, 0x9f, 0x08, 0x3c, 0x2e, 0xe5, 0x99, 0x69, 0x1e, 0xf4, 0xbc, 0x68, 0xd5, + 0x72, 0x30, 0x3f, 0x63, 0xfa, 0xcb, 0xd3, 0xc5, 0x94, 0xe4, 0x25, 0x68, 0x8c, 0xb2, 0xfc, 0x89, + 0x9f, 0xce, 0xf2, 0x5f, 0x87, 0xb9, 0xae, 0x85, 0xf7, 0xb0, 0xa3, 0xee, 0xcb, 0x46, 0xaf, 0xc3, + 0xcf, 0xef, 0x64, 0x5d, 0xda, 0x56, 0xaf, 0x83, 0xee, 0x41, 0xd1, 0x2b, 0xc2, 0x31, 0xba, 0x9b, + 0xc0, 0xcb, 0xa5, 0x73, 0x44, 0x2f, 0xfe, 0x2f, 0x01, 0x16, 0x42, 0x6d, 0xe2, 0x73, 0x62, 0x1d, + 0xb2, 0xbe, 0x39, 0xb0, 0x4b, 0xc2, 0x39, 0x63, 0x37, 0x83, 0xcc, 0x48, 0x86, 0x4b, 0xee, 0x63, + 0x69, 0xf6, 0x7e, 0x5f, 0x6c, 0xe2, 0x9c, 0x62, 0x2f, 0xfa, 0x72, 0xd6, 0x02, 0x0f, 0xf0, 0x26, + 0x49, 0x72, 0xa2, 0x49, 0x22, 0xfe, 0x96, 0x00, 0x45, 0xfa, 0x80, 0x27, 0x18, 0x6b, 0xb1, 0x58, + 0x27, 0x37, 0xfc, 0x3b, 0x31, 0xf9, 0x09, 0x9d, 0xd0, 0x17, 0x47, 0x92, 0xe1, 0x2f, 0x8e, 0x88, + 0x3f, 0x14, 0x20, 0xef, 0xd5, 0x90, 0x7d, 0x77, 0x6f, 0x44, 0x1a, 0xd0, 0x37, 0xfb, 0xe6, 0x9c, + 0x9b, 0xef, 0x64, 0xa2, 0x4f, 0x01, 0x06, 0xf3, 0x9d, 0xb0, 0x6f, 0xa5, 0xfd, 0x4d, 0x77, 0xe4, + 0x90, 0x2a, 0x56, 0xfc, 0xbc, 0x16, 0x6f, 0x70, 0x58, 0x49, 0xa2, 0x9f, 0x2c, 0x35, 0xdb, 0x87, + 0x2c, 0xc5, 0xcc, 0x44, 0x66, 0x0b, 0xf1, 0x68, 0x2c, 0xe0, 0xbb, 0x79, 0x5a, 0xb3, 0x41, 0x3f, + 0x66, 0xca, 0x7e, 0xdb, 0xe2, 0x93, 0x80, 0x02, 0x69, 0xe7, 0x13, 0x2d, 0x4d, 0x64, 0x4a, 0x5d, + 0x2d, 0xb1, 0xb1, 0xf2, 0xc7, 0xc1, 0x9e, 0xa8, 0x1e, 0x12, 0x14, 0xf7, 0x08, 0x92, 0x87, 0x4a, + 0x7b, 0x54, 0x14, 0x5a, 0xa8, 0xe7, 0x24, 0x52, 0x1a, 0x3d, 0x09, 0xa5, 0x03, 0x49, 0x0c, 0x47, + 0x1c, 0x83, 0x2a, 0x0d, 0xa5, 0x0d, 0xf9, 0x30, 0x3c, 0xd6, 0x47, 0x3e, 0x3e, 0x38, 0xe8, 0x3f, + 0x4e, 0xfd, 0xe8, 0x87, 0x8b, 0x82, 0xf8, 0x09, 0x20, 0x09, 0xdb, 0xd8, 0x79, 0xd1, 0x33, 0x2d, + 0x3f, 0xb5, 0xca, 0xed, 0xbe, 0x4f, 0xb0, 0x4c, 0xaf, 0x66, 0xcf, 0xa2, 0x92, 0x27, 0x5d, 0x84, + 0x85, 0x10, 0x37, 0x33, 0x16, 0xe2, 0x87, 0x70, 0xe5, 0xa9, 0x69, 0xdb, 0x7a, 0x97, 0x40, 0x57, + 0x3a, 0x2b, 0xc9, 0xd2, 0xe0, 0x99, 0xc7, 0x74, 0x97, 0x6e, 0x62, 0x18, 0xcc, 0x8c, 0x64, 0x24, + 0xef, 0x5a, 0xfc, 0x03, 0x01, 0x2e, 0x0f, 0x72, 0x32, 0x2d, 0x47, 0x9d, 0xad, 0x9c, 0x55, 0x4d, + 0x3f, 0x93, 0xde, 0xf8, 0xd1, 0xea, 0x16, 0x27, 0x2e, 0x26, 0x7f, 0xa6, 0xdc, 0x51, 0xa8, 0xf9, + 0xe0, 0xe7, 0xbc, 0xf3, 0x9c, 0xbc, 0xc9, 0xa8, 0xbe, 0x25, 0x49, 0x4d, 0x66, 0x49, 0x9a, 0x50, + 0x58, 0x37, 0x75, 0x83, 0x78, 0xb2, 0x6e, 0x7b, 0x57, 0x20, 0xbf, 0xab, 0x1b, 0x8a, 0x75, 0x22, + 0xbb, 0x07, 0x30, 0x84, 0x71, 0x07, 0x30, 0xa4, 0x1c, 0xe3, 0xe0, 0x97, 0xe2, 0x8f, 0x05, 0x28, + 0xfa, 0x62, 0xb9, 0x45, 0x7e, 0x17, 0x40, 0x6d, 0xf7, 0x6c, 0x07, 0x5b, 0x6e, 0x2f, 0xcd, 0xb1, + 0xe0, 0xf1, 0x0a, 0xa3, 0xd6, 0xd7, 0xa4, 0x0c, 0x2f, 0x50, 0xd7, 0xd0, 0x8d, 0x70, 0x1a, 0x8a, + 0xe9, 0x55, 0x38, 0x1b, 0x48, 0x3e, 0x41, 0xba, 0xdd, 0x76, 0x4c, 0xcb, 0xc3, 0x4c, 0xbc, 0xdb, + 0xdd, 0xa4, 0xf5, 0xf4, 0xf4, 0x34, 0x29, 0xb7, 0x02, 0x79, 0xb2, 0xdc, 0x1f, 0x62, 0xaf, 0x49, + 0xa9, 0xf1, 0x4d, 0x62, 0x1c, 0x6e, 0x93, 0x7e, 0x97, 0x38, 0xa9, 0xac, 0x37, 0xbc, 0x1e, 0x1e, + 0x61, 0xd1, 0x7e, 0x26, 0x98, 0xfc, 0x71, 0xb2, 0xf4, 0x98, 0xd4, 0x13, 0xf9, 0x16, 0xa4, 0xdd, + 0x8f, 0x41, 0xf3, 0x09, 0x72, 0x65, 0x89, 0x7d, 0x2d, 0x7a, 0xc9, 0xfd, 0x5a, 0xf4, 0xd2, 0x1a, + 0x2f, 0xc0, 0xcc, 0xf8, 0x0f, 0xfe, 0xcb, 0xa2, 0x20, 0x79, 0x4c, 0xf7, 0x1b, 0x64, 0x84, 0x0f, + 0xac, 0xc2, 0x28, 0x0f, 0x10, 0xf8, 0x5a, 0x0f, 0xff, 0x3e, 0xf0, 0xca, 0x9a, 0xbc, 0xb3, 0x55, + 0x79, 0xbe, 0xb9, 0x59, 0x6f, 0x36, 0xab, 0x6b, 0x45, 0x01, 0x15, 0x61, 0x2e, 0xf4, 0xad, 0x9f, + 0x04, 0xfb, 0x62, 0xf0, 0xfd, 0x9f, 0x01, 0xf0, 0x3f, 0x1b, 0x46, 0x64, 0x6d, 0x54, 0x3f, 0x93, + 0x5f, 0xae, 0x3c, 0xdb, 0xa9, 0x36, 0x8a, 0x53, 0x08, 0x41, 0x7e, 0x75, 0xa5, 0x59, 0xa9, 0xc9, + 0x52, 0xb5, 0xb1, 0xfd, 0x7c, 0xab, 0x51, 0x75, 0xbf, 0x34, 0x7c, 0x7f, 0x0d, 0xe6, 0x82, 0x89, + 0x85, 0xd0, 0x02, 0x14, 0x2a, 0xb5, 0x6a, 0x65, 0x43, 0x7e, 0x59, 0x5f, 0x91, 0x5f, 0xec, 0x54, + 0x77, 0xaa, 0xc5, 0x29, 0x5a, 0x35, 0x4a, 0x7c, 0xb2, 0xf3, 0xec, 0x59, 0x51, 0x40, 0x05, 0xc8, + 0xb2, 0x6b, 0xfa, 0x5d, 0xa0, 0x62, 0xe2, 0xfe, 0x26, 0x64, 0x03, 0x09, 0x84, 0xc9, 0xe3, 0xb6, + 0x77, 0x1a, 0x35, 0xb9, 0x59, 0xdf, 0xac, 0x36, 0x9a, 0x2b, 0x9b, 0xdb, 0x4c, 0x06, 0xa5, 0xad, + 0xac, 0x3e, 0x97, 0x9a, 0x45, 0xc1, 0xbb, 0x6e, 0x3e, 0xdf, 0xa9, 0xd4, 0xdc, 0x66, 0x88, 0xa9, + 0x74, 0xb2, 0x98, 0xbc, 0xff, 0x97, 0x04, 0xb8, 0x3c, 0x24, 0xc9, 0x0e, 0xca, 0xc2, 0xec, 0x8e, + 0x41, 0xb3, 0xa9, 0x16, 0xa7, 0x50, 0x2e, 0x90, 0x67, 0xa7, 0x28, 0xa0, 0x34, 0xcb, 0x74, 0x52, + 0x4c, 0xa0, 0x19, 0x48, 0x34, 0x1e, 0x15, 0x93, 0xa4, 0xa6, 0x81, 0x34, 0x35, 0xc5, 0x14, 0xca, + 0xf0, 0x04, 0x19, 0xc5, 0x69, 0x34, 0xe7, 0xe7, 0xa9, 0x28, 0xce, 0x10, 0x51, 0x5e, 0xbe, 0x87, + 0xe2, 0xec, 0xfd, 0xeb, 0x10, 0x38, 0x53, 0x8f, 0x00, 0x66, 0x9e, 0x29, 0x0e, 0xb6, 0x9d, 0xe2, + 0x14, 0x9a, 0x85, 0xe4, 0x4a, 0xbb, 0x5d, 0x14, 0x1e, 0xfe, 0x8b, 0x14, 0xa4, 0xdd, 0xef, 0xdf, + 0xa0, 0x67, 0x30, 0xcd, 0xb6, 0xae, 0x17, 0x87, 0x7b, 0xf6, 0x74, 0xf2, 0x96, 0xaf, 0x8d, 0x73, + 0xfd, 0xc5, 0x29, 0xf4, 0xff, 0x41, 0x36, 0xe0, 0x31, 0xa1, 0xa1, 0xdb, 0x6f, 0x21, 0x2f, 0xb1, + 0x7c, 0x7b, 0x5c, 0x31, 0x4f, 0xfe, 0x2b, 0xc8, 0x78, 0x16, 0x1c, 0xdd, 0x18, 0x65, 0xdf, 0x5d, + 0xd9, 0xa3, 0x17, 0x01, 0x32, 0xd7, 0xc4, 0xa9, 0xf7, 0x05, 0x64, 0x01, 0x1a, 0x34, 0xb6, 0x28, + 0x2a, 0x20, 0x62, 0xa8, 0x35, 0x2f, 0xdf, 0x9f, 0xa8, 0xb4, 0xff, 0x4c, 0xa2, 0x2c, 0x7f, 0xc5, + 0x88, 0x56, 0xd6, 0xc0, 0x7a, 0x14, 0xad, 0xac, 0x88, 0x85, 0x67, 0x0a, 0xbd, 0x80, 0x14, 0xb1, + 0x94, 0x28, 0xca, 0x87, 0xec, 0xb3, 0xcc, 0xe5, 0x1b, 0x23, 0xcb, 0xb8, 0x22, 0x57, 0xef, 0xfd, + 0xe8, 0xcf, 0xaf, 0x4e, 0xfd, 0xe8, 0xec, 0xaa, 0xf0, 0xe3, 0xb3, 0xab, 0xc2, 0x9f, 0x9e, 0x5d, + 0x15, 0xfe, 0xec, 0xec, 0xaa, 0xf0, 0xfd, 0x9f, 0x5c, 0x9d, 0xfa, 0xf1, 0x4f, 0xae, 0x4e, 0xfd, + 0xe9, 0x4f, 0xae, 0x4e, 0x7d, 0x3e, 0xcb, 0xb9, 0x77, 0x67, 0xa8, 0x51, 0x79, 0xf4, 0x7f, 0x03, + 0x00, 0x00, 0xff, 0xff, 0xc9, 0x4d, 0x99, 0x39, 0xbd, 0x7f, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index ab86df7a67b8..84abefcb99cf 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -1488,6 +1488,26 @@ message AdminScatterResponse { repeated RangeInfo range_infos = 3 [(gogoproto.nullable) = false]; } +// 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]; +} + // AdminVerifyProtectedTimestampRequest is the argument to the // AdminVerifyProtectedTimestamp method which ensures that the specified record // will be seen before data can be garbage collected at the timestamp. @@ -1717,6 +1737,7 @@ message RequestUnion { SubsumeRequest subsume = 43; RangeStatsRequest range_stats = 44; AdminVerifyProtectedTimestampRequest admin_verify_protected_timestamp = 49; + MigrateRequest migrate = 50; } reserved 8, 15, 23, 25, 27; } @@ -1768,6 +1789,7 @@ message ResponseUnion { SubsumeResponse subsume = 43; RangeStatsResponse range_stats = 44; AdminVerifyProtectedTimestampResponse admin_verify_protected_timestamp = 49; + MigrateResponse migrate = 50; } reserved 8, 15, 23, 25, 27, 28; } diff --git a/pkg/roachpb/batch_generated.go b/pkg/roachpb/batch_generated.go index 5f0b865c6e3c..99556c74f790 100644 --- a/pkg/roachpb/batch_generated.go +++ b/pkg/roachpb/batch_generated.go @@ -164,6 +164,8 @@ func (ru RequestUnion) GetInner() Request { return t.RangeStats case *RequestUnion_AdminVerifyProtectedTimestamp: return t.AdminVerifyProtectedTimestamp + case *RequestUnion_Migrate: + return t.Migrate default: return nil } @@ -258,6 +260,8 @@ func (ru ResponseUnion) GetInner() Response { return t.RangeStats case *ResponseUnion_AdminVerifyProtectedTimestamp: return t.AdminVerifyProtectedTimestamp + case *ResponseUnion_Migrate: + return t.Migrate default: return nil } @@ -423,6 +427,8 @@ func (ru *RequestUnion) MustSetInner(r Request) { union = &RequestUnion_RangeStats{t} case *AdminVerifyProtectedTimestampRequest: union = &RequestUnion_AdminVerifyProtectedTimestamp{t} + case *MigrateRequest: + union = &RequestUnion_Migrate{t} default: panic(fmt.Sprintf("unsupported type %T for %T", r, ru)) } @@ -520,13 +526,15 @@ func (ru *ResponseUnion) MustSetInner(r Response) { union = &ResponseUnion_RangeStats{t} case *AdminVerifyProtectedTimestampResponse: union = &ResponseUnion_AdminVerifyProtectedTimestamp{t} + case *MigrateResponse: + union = &ResponseUnion_Migrate{t} default: panic(fmt.Sprintf("unsupported type %T for %T", r, ru)) } ru.Value = union } -type reqCounts [44]int32 +type reqCounts [45]int32 // getReqCounts returns the number of times each // request type appears in the batch. @@ -622,6 +630,8 @@ func (ba *BatchRequest) getReqCounts() reqCounts { counts[42]++ case *RequestUnion_AdminVerifyProtectedTimestamp: counts[43]++ + case *RequestUnion_Migrate: + counts[44]++ default: panic(fmt.Sprintf("unsupported request: %+v", ru)) } @@ -674,6 +684,7 @@ var requestNames = []string{ "Subsume", "RngStats", "AdmVerifyProtectedTimestamp", + "Migrate", } // Summary prints a short summary of the requests in a batch. @@ -881,6 +892,10 @@ type adminVerifyProtectedTimestampResponseAlloc struct { union ResponseUnion_AdminVerifyProtectedTimestamp resp AdminVerifyProtectedTimestampResponse } +type migrateResponseAlloc struct { + union ResponseUnion_Migrate + resp MigrateResponse +} // CreateReply creates replies for each of the contained requests, wrapped in a // BatchResponse. The response objects are batch allocated to minimize @@ -935,6 +950,7 @@ func (ba *BatchRequest) CreateReply() *BatchResponse { var buf41 []subsumeResponseAlloc var buf42 []rangeStatsResponseAlloc var buf43 []adminVerifyProtectedTimestampResponseAlloc + var buf44 []migrateResponseAlloc for i, r := range ba.Requests { switch r.GetValue().(type) { @@ -1246,6 +1262,13 @@ func (ba *BatchRequest) CreateReply() *BatchResponse { buf43[0].union.AdminVerifyProtectedTimestamp = &buf43[0].resp br.Responses[i].Value = &buf43[0].union buf43 = buf43[1:] + case *RequestUnion_Migrate: + if buf44 == nil { + buf44 = make([]migrateResponseAlloc, counts[44]) + } + buf44[0].union.Migrate = &buf44[0].resp + br.Responses[i].Value = &buf44[0].union + buf44 = buf44[1:] default: panic(fmt.Sprintf("unsupported request: %+v", r)) } @@ -1344,6 +1367,8 @@ func CreateRequest(method Method) Request { return &RangeStatsRequest{} case AdminVerifyProtectedTimestamp: return &AdminVerifyProtectedTimestampRequest{} + case Migrate: + return &MigrateRequest{} default: panic(fmt.Sprintf("unsupported method: %+v", method)) } diff --git a/pkg/roachpb/method.go b/pkg/roachpb/method.go index baaeb32cf080..4acb8e29916c 100644 --- a/pkg/roachpb/method.go +++ b/pkg/roachpb/method.go @@ -158,6 +158,9 @@ const ( // VerifyProtectedTimestamp determines whether the specified protection record // will be respected by this Range. AdminVerifyProtectedTimestamp + // 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 // NumMethods represents the total number of API methods. NumMethods ) diff --git a/pkg/roachpb/method_string.go b/pkg/roachpb/method_string.go index 67c7b3157110..402d2d7e595a 100644 --- a/pkg/roachpb/method_string.go +++ b/pkg/roachpb/method_string.go @@ -52,12 +52,13 @@ func _() { _ = x[Subsume-41] _ = x[RangeStats-42] _ = x[AdminVerifyProtectedTimestamp-43] - _ = x[NumMethods-44] + _ = x[Migrate-44] + _ = x[NumMethods-45] } -const _Method_name = "GetPutConditionalPutIncrementDeleteDeleteRangeClearRangeRevertRangeScanReverseScanEndTxnAdminSplitAdminUnsplitAdminMergeAdminTransferLeaseAdminChangeReplicasAdminRelocateRangeHeartbeatTxnGCPushTxnRecoverTxnQueryTxnQueryIntentResolveIntentResolveIntentRangeMergeTruncateLogRequestLeaseTransferLeaseLeaseInfoComputeChecksumCheckConsistencyInitPutWriteBatchExportImportAdminScatterAddSSTableRecomputeStatsRefreshRefreshRangeSubsumeRangeStatsAdminVerifyProtectedTimestampNumMethods" +const _Method_name = "GetPutConditionalPutIncrementDeleteDeleteRangeClearRangeRevertRangeScanReverseScanEndTxnAdminSplitAdminUnsplitAdminMergeAdminTransferLeaseAdminChangeReplicasAdminRelocateRangeHeartbeatTxnGCPushTxnRecoverTxnQueryTxnQueryIntentResolveIntentResolveIntentRangeMergeTruncateLogRequestLeaseTransferLeaseLeaseInfoComputeChecksumCheckConsistencyInitPutWriteBatchExportImportAdminScatterAddSSTableRecomputeStatsRefreshRefreshRangeSubsumeRangeStatsAdminVerifyProtectedTimestampMigrateNumMethods" -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, 402, 409, 421, 428, 438, 467, 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 ccb08e278862..e1796b8d2616 100644 --- a/pkg/server/init.go +++ b/pkg/server/init.go @@ -541,7 +541,7 @@ func (s *initServer) initializeFirstStoreAfterJoin( type initServerCfg struct { advertiseAddr string binaryMinSupportedVersion roachpb.Version - binaryVersion roachpb.Version // This is what's used for bootstrap. + binaryVersion roachpb.Version // the version used during bootstrap defaultSystemZoneConfig zonepb.ZoneConfig defaultZoneConfig zonepb.ZoneConfig diff --git a/pkg/server/migration.go b/pkg/server/migration.go index d5f2252dab5b..efdc009c2241 100644 --- a/pkg/server/migration.go +++ b/pkg/server/migration.go @@ -81,6 +81,9 @@ func (m *migrationServer) ValidateTargetClusterVersion( func (m *migrationServer) BumpClusterVersion( ctx context.Context, req *serverpb.BumpClusterVersionRequest, ) (*serverpb.BumpClusterVersionResponse, error) { + ctx, span := m.server.AnnotateCtxWithSpan(ctx, "bump-cv") + defer span.Finish() + m.Lock() defer m.Unlock() @@ -98,6 +101,7 @@ func (m *migrationServer) BumpClusterVersion( if err := func() error { if !prevCV.Less(newCV.Version) { // Nothing to do. + log.Infof(ctx, "xxx: here %s %s %s %s", prevCV, newCV, versionSetting.BinaryMinSupportedVersion(), versionSetting.BinaryVersion()) return nil } @@ -134,3 +138,31 @@ func (m *migrationServer) BumpClusterVersion( resp := &serverpb.BumpClusterVersionResponse{} return resp, nil } + +// FlushAllEngines implements the MigrationServer interface. +func (m *migrationServer) FlushAllEngines( + _ context.Context, _ *serverpb.FlushAllEnginesRequest, +) (*serverpb.FlushAllEnginesResponse, error) { + for _, eng := range m.server.engines { + if err := eng.Flush(); err != nil { + return nil, err + } + } + + resp := &serverpb.FlushAllEnginesResponse{} + return resp, nil +} + +// GCReplicas implements the MigrationServer interface. +func (m *migrationServer) GCReplicas( + _ context.Context, _ *serverpb.GCReplicasRequest, +) (*serverpb.GCReplicasResponse, error) { + if err := m.server.node.stores.VisitStores(func(s *kvserver.Store) error { + return s.GCReplicas() + }); err != nil { + return nil, err + } + + resp := &serverpb.GCReplicasResponse{} + return resp, nil +} diff --git a/pkg/server/migration_test.go b/pkg/server/migration_test.go index 0056a4d8985e..9010c5745913 100644 --- a/pkg/server/migration_test.go +++ b/pkg/server/migration_test.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -96,3 +97,122 @@ func TestValidateTargetClusterVersion(t *testing.T) { s.Stopper().Stop(context.Background()) } } + +func TestBumpClusterVersion(t *testing.T) { + defer leaktest.AfterTest(t)() + + v := func(major, minor int32) roachpb.Version { + return roachpb.Version{Major: major, Minor: minor} + } + cv := func(major, minor int32) clusterversion.ClusterVersion { + return clusterversion.ClusterVersion{Version: v(major, minor)} + } + + var tests = []struct { + binaryVersion roachpb.Version + activeClusterVersion clusterversion.ClusterVersion // akin to min supported binary version + bumpClusterVersion clusterversion.ClusterVersion + expClusterVersion clusterversion.ClusterVersion + }{ + { + binaryVersion: v(20, 2), + activeClusterVersion: cv(20, 1), + bumpClusterVersion: cv(20, 1), + expClusterVersion: cv(20, 1), + }, + { + binaryVersion: v(20, 2), + activeClusterVersion: cv(20, 1), + bumpClusterVersion: cv(20, 2), + expClusterVersion: cv(20, 2), + }, + { + binaryVersion: v(20, 2), + activeClusterVersion: cv(20, 2), + bumpClusterVersion: cv(20, 1), + expClusterVersion: cv(20, 2), + }, + } + + ctx := context.Background() + for _, test := range tests { + st := cluster.MakeTestingClusterSettingsWithVersions( + test.binaryVersion, + test.activeClusterVersion.Version, + false, /* initializeVersion */ + ) + + s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ + Settings: st, + Knobs: base.TestingKnobs{ + Server: &TestingKnobs{ + // This test wants to bootstrap at the previously active + // cluster version, so we can actually bump the cluster + // version to the binary version. Think a cluster with + // active cluster version v20.1, but running v20.2 binaries. + BinaryVersionOverride: test.activeClusterVersion.Version, + }, + }, + }) + + migrationServer := s.MigrationServer().(*migrationServer) + req := &serverpb.BumpClusterVersionRequest{ + ClusterVersion: &test.bumpClusterVersion, + } + if _, err := migrationServer.BumpClusterVersion(ctx, req); err != nil { + t.Fatal(err) + } + + // Check to see our post-bump active cluster version is what we expect. + if got := s.ClusterSettings().Version.ActiveVersion(ctx); got != test.expClusterVersion { + t.Fatalf("expected active cluster version %s, got %s", test.expClusterVersion, got) + } + + // Check to see that our bumped cluster version was persisted to disk. + synthesizedCV, err := kvserver.SynthesizeClusterVersionFromEngines( + ctx, s.Engines(), test.binaryVersion, + test.activeClusterVersion.Version, + ) + if err != nil { + t.Fatal(err) + } + if synthesizedCV != test.expClusterVersion { + t.Fatalf("expected synthesized cluster version %s, got %s", test.expClusterVersion, synthesizedCV) + } + + s.Stopper().Stop(context.Background()) + } +} + +func TestMigrationGCReplicas(t *testing.T) { + defer leaktest.AfterTest(t)() + + const numStores = 3 + var storeSpecs []base.StoreSpec + for i := 0; i < numStores; i++ { + storeSpecs = append(storeSpecs, base.StoreSpec{InMemory: true}) + } + + intercepted := 0 + s, _, _ := serverutils.StartServer(t, base.TestServerArgs{ + StoreSpecs: storeSpecs, + Knobs: base.TestingKnobs{ + Store: &kvserver.StoreTestingKnobs{ + GCReplicasInterceptor: func() { + intercepted++ + }, + }, + }, + }) + + migrationServer := s.MigrationServer().(*migrationServer) + if _, err := migrationServer.GCReplicas(context.Background(), &serverpb.GCReplicasRequest{}); err != nil { + t.Fatal(err) + } + + if intercepted != numStores { + t.Fatalf("expected to have GC-ed replicas on %d stores, found %d", numStores, intercepted) + } + + s.Stopper().Stop(context.Background()) +} diff --git a/pkg/server/serverpb/migration.pb.go b/pkg/server/serverpb/migration.pb.go index e2eba51c12f3..82166317327e 100644 --- a/pkg/server/serverpb/migration.pb.go +++ b/pkg/server/serverpb/migration.pb.go @@ -36,7 +36,7 @@ func (m *ValidateTargetClusterVersionRequest) Reset() { *m = ValidateTar func (m *ValidateTargetClusterVersionRequest) String() string { return proto.CompactTextString(m) } func (*ValidateTargetClusterVersionRequest) ProtoMessage() {} func (*ValidateTargetClusterVersionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_migration_8dfeb6fcf9144e4c, []int{0} + return fileDescriptor_migration_a15deb200006b6c7, []int{0} } func (m *ValidateTargetClusterVersionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -70,7 +70,7 @@ func (m *ValidateTargetClusterVersionResponse) Reset() { *m = ValidateTa func (m *ValidateTargetClusterVersionResponse) String() string { return proto.CompactTextString(m) } func (*ValidateTargetClusterVersionResponse) ProtoMessage() {} func (*ValidateTargetClusterVersionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_migration_8dfeb6fcf9144e4c, []int{1} + return fileDescriptor_migration_a15deb200006b6c7, []int{1} } func (m *ValidateTargetClusterVersionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -95,8 +95,8 @@ func (m *ValidateTargetClusterVersionResponse) XXX_DiscardUnknown() { var xxx_messageInfo_ValidateTargetClusterVersionResponse proto.InternalMessageInfo -// BumpClusterVersionRequest is used to inform a given node of a cluster version -// bump. +// BumpClusterVersionRequest is used to inform the tharget node of a cluster +// version bump. type BumpClusterVersionRequest struct { ClusterVersion *clusterversion.ClusterVersion `protobuf:"bytes,1,opt,name=cluster_version,json=clusterVersion,proto3" json:"cluster_version,omitempty"` } @@ -105,7 +105,7 @@ func (m *BumpClusterVersionRequest) Reset() { *m = BumpClusterVersionReq func (m *BumpClusterVersionRequest) String() string { return proto.CompactTextString(m) } func (*BumpClusterVersionRequest) ProtoMessage() {} func (*BumpClusterVersionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_migration_8dfeb6fcf9144e4c, []int{2} + return fileDescriptor_migration_a15deb200006b6c7, []int{2} } func (m *BumpClusterVersionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -138,7 +138,7 @@ func (m *BumpClusterVersionResponse) Reset() { *m = BumpClusterVersionRe func (m *BumpClusterVersionResponse) String() string { return proto.CompactTextString(m) } func (*BumpClusterVersionResponse) ProtoMessage() {} func (*BumpClusterVersionResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_migration_8dfeb6fcf9144e4c, []int{3} + return fileDescriptor_migration_a15deb200006b6c7, []int{3} } func (m *BumpClusterVersionResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -163,11 +163,149 @@ func (m *BumpClusterVersionResponse) XXX_DiscardUnknown() { var xxx_messageInfo_BumpClusterVersionResponse proto.InternalMessageInfo +// GCReplicasRequest is used to instruct the target node to process all GC-able +// replicas. +type GCReplicasRequest struct { +} + +func (m *GCReplicasRequest) Reset() { *m = GCReplicasRequest{} } +func (m *GCReplicasRequest) String() string { return proto.CompactTextString(m) } +func (*GCReplicasRequest) ProtoMessage() {} +func (*GCReplicasRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_migration_a15deb200006b6c7, []int{4} +} +func (m *GCReplicasRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GCReplicasRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GCReplicasRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GCReplicasRequest.Merge(dst, src) +} +func (m *GCReplicasRequest) XXX_Size() int { + return m.Size() +} +func (m *GCReplicasRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GCReplicasRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_GCReplicasRequest proto.InternalMessageInfo + +// GCReplicasResponse is the response to a GCReplicasRequest. +type GCReplicasResponse struct { +} + +func (m *GCReplicasResponse) Reset() { *m = GCReplicasResponse{} } +func (m *GCReplicasResponse) String() string { return proto.CompactTextString(m) } +func (*GCReplicasResponse) ProtoMessage() {} +func (*GCReplicasResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_migration_a15deb200006b6c7, []int{5} +} +func (m *GCReplicasResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GCReplicasResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GCReplicasResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_GCReplicasResponse.Merge(dst, src) +} +func (m *GCReplicasResponse) XXX_Size() int { + return m.Size() +} +func (m *GCReplicasResponse) XXX_DiscardUnknown() { + xxx_messageInfo_GCReplicasResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_GCReplicasResponse proto.InternalMessageInfo + +// FlushAllEnginesRequest is used to instruct the target node to flush all its +// engines. +type FlushAllEnginesRequest struct { +} + +func (m *FlushAllEnginesRequest) Reset() { *m = FlushAllEnginesRequest{} } +func (m *FlushAllEnginesRequest) String() string { return proto.CompactTextString(m) } +func (*FlushAllEnginesRequest) ProtoMessage() {} +func (*FlushAllEnginesRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_migration_a15deb200006b6c7, []int{6} +} +func (m *FlushAllEnginesRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *FlushAllEnginesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *FlushAllEnginesRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_FlushAllEnginesRequest.Merge(dst, src) +} +func (m *FlushAllEnginesRequest) XXX_Size() int { + return m.Size() +} +func (m *FlushAllEnginesRequest) XXX_DiscardUnknown() { + xxx_messageInfo_FlushAllEnginesRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_FlushAllEnginesRequest proto.InternalMessageInfo + +// FlushAllEnginesResponse is the response to a FlushAllEnginesRequest. +type FlushAllEnginesResponse struct { +} + +func (m *FlushAllEnginesResponse) Reset() { *m = FlushAllEnginesResponse{} } +func (m *FlushAllEnginesResponse) String() string { return proto.CompactTextString(m) } +func (*FlushAllEnginesResponse) ProtoMessage() {} +func (*FlushAllEnginesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_migration_a15deb200006b6c7, []int{7} +} +func (m *FlushAllEnginesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *FlushAllEnginesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *FlushAllEnginesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_FlushAllEnginesResponse.Merge(dst, src) +} +func (m *FlushAllEnginesResponse) XXX_Size() int { + return m.Size() +} +func (m *FlushAllEnginesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_FlushAllEnginesResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_FlushAllEnginesResponse proto.InternalMessageInfo + func init() { proto.RegisterType((*ValidateTargetClusterVersionRequest)(nil), "cockroach.server.serverpb.ValidateTargetClusterVersionRequest") proto.RegisterType((*ValidateTargetClusterVersionResponse)(nil), "cockroach.server.serverpb.ValidateTargetClusterVersionResponse") proto.RegisterType((*BumpClusterVersionRequest)(nil), "cockroach.server.serverpb.BumpClusterVersionRequest") proto.RegisterType((*BumpClusterVersionResponse)(nil), "cockroach.server.serverpb.BumpClusterVersionResponse") + proto.RegisterType((*GCReplicasRequest)(nil), "cockroach.server.serverpb.GCReplicasRequest") + proto.RegisterType((*GCReplicasResponse)(nil), "cockroach.server.serverpb.GCReplicasResponse") + proto.RegisterType((*FlushAllEnginesRequest)(nil), "cockroach.server.serverpb.FlushAllEnginesRequest") + proto.RegisterType((*FlushAllEnginesResponse)(nil), "cockroach.server.serverpb.FlushAllEnginesResponse") } // Reference imports to suppress errors if they are not otherwise used. @@ -188,7 +326,7 @@ type MigrationClient interface { // // node's minimum supported version <= version <= node's binary version ValidateTargetClusterVersion(ctx context.Context, in *ValidateTargetClusterVersionRequest, opts ...grpc.CallOption) (*ValidateTargetClusterVersionResponse, error) - // BumpClusterVersion is used to inform a given node of a cluster version + // BumpClusterVersion is used to inform the target node of a cluster version // bump. The node is responsible for durably persisting the message and // enabling the corresponding version gates. // @@ -196,6 +334,12 @@ type MigrationClient interface { // which checks to see that all nodes in the cluster are running binaries // that would be able to support the intended version bump. BumpClusterVersion(ctx context.Context, in *BumpClusterVersionRequest, opts ...grpc.CallOption) (*BumpClusterVersionResponse, error) + // FlushAllEngines is used to instruct the target node to flush all its + // engines. + FlushAllEngines(ctx context.Context, in *FlushAllEnginesRequest, opts ...grpc.CallOption) (*FlushAllEnginesResponse, error) + // GCReplicas is used to instruct the target node to process all GC-able + // replicas. + GCReplicas(ctx context.Context, in *GCReplicasRequest, opts ...grpc.CallOption) (*GCReplicasResponse, error) } type migrationClient struct { @@ -224,6 +368,24 @@ func (c *migrationClient) BumpClusterVersion(ctx context.Context, in *BumpCluste return out, nil } +func (c *migrationClient) FlushAllEngines(ctx context.Context, in *FlushAllEnginesRequest, opts ...grpc.CallOption) (*FlushAllEnginesResponse, error) { + out := new(FlushAllEnginesResponse) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Migration/FlushAllEngines", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *migrationClient) GCReplicas(ctx context.Context, in *GCReplicasRequest, opts ...grpc.CallOption) (*GCReplicasResponse, error) { + out := new(GCReplicasResponse) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Migration/GCReplicas", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // MigrationServer is the server API for Migration service. type MigrationServer interface { // ValidateTargetClusterVersion is used to verify that the target node is @@ -232,7 +394,7 @@ type MigrationServer interface { // // node's minimum supported version <= version <= node's binary version ValidateTargetClusterVersion(context.Context, *ValidateTargetClusterVersionRequest) (*ValidateTargetClusterVersionResponse, error) - // BumpClusterVersion is used to inform a given node of a cluster version + // BumpClusterVersion is used to inform the target node of a cluster version // bump. The node is responsible for durably persisting the message and // enabling the corresponding version gates. // @@ -240,6 +402,12 @@ type MigrationServer interface { // which checks to see that all nodes in the cluster are running binaries // that would be able to support the intended version bump. BumpClusterVersion(context.Context, *BumpClusterVersionRequest) (*BumpClusterVersionResponse, error) + // FlushAllEngines is used to instruct the target node to flush all its + // engines. + FlushAllEngines(context.Context, *FlushAllEnginesRequest) (*FlushAllEnginesResponse, error) + // GCReplicas is used to instruct the target node to process all GC-able + // replicas. + GCReplicas(context.Context, *GCReplicasRequest) (*GCReplicasResponse, error) } func RegisterMigrationServer(s *grpc.Server, srv MigrationServer) { @@ -282,6 +450,42 @@ func _Migration_BumpClusterVersion_Handler(srv interface{}, ctx context.Context, return interceptor(ctx, in, info, handler) } +func _Migration_FlushAllEngines_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(FlushAllEnginesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(MigrationServer).FlushAllEngines(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/cockroach.server.serverpb.Migration/FlushAllEngines", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(MigrationServer).FlushAllEngines(ctx, req.(*FlushAllEnginesRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _Migration_GCReplicas_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GCReplicasRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(MigrationServer).GCReplicas(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/cockroach.server.serverpb.Migration/GCReplicas", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(MigrationServer).GCReplicas(ctx, req.(*GCReplicasRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _Migration_serviceDesc = grpc.ServiceDesc{ ServiceName: "cockroach.server.serverpb.Migration", HandlerType: (*MigrationServer)(nil), @@ -294,6 +498,14 @@ var _Migration_serviceDesc = grpc.ServiceDesc{ MethodName: "BumpClusterVersion", Handler: _Migration_BumpClusterVersion_Handler, }, + { + MethodName: "FlushAllEngines", + Handler: _Migration_FlushAllEngines_Handler, + }, + { + MethodName: "GCReplicas", + Handler: _Migration_GCReplicas_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "server/serverpb/migration.proto", @@ -391,6 +603,78 @@ func (m *BumpClusterVersionResponse) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *GCReplicasRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GCReplicasRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + +func (m *GCReplicasResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *GCReplicasResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + +func (m *FlushAllEnginesRequest) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *FlushAllEnginesRequest) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + +func (m *FlushAllEnginesResponse) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *FlushAllEnginesResponse) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + func encodeVarintMigration(dAtA []byte, offset int, v uint64) int { for v >= 1<<7 { dAtA[offset] = uint8(v&0x7f | 0x80) @@ -444,6 +728,42 @@ func (m *BumpClusterVersionResponse) Size() (n int) { return n } +func (m *GCReplicasRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + +func (m *GCReplicasResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + +func (m *FlushAllEnginesRequest) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + +func (m *FlushAllEnginesResponse) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + func sovMigration(x uint64) (n int) { for { n++ @@ -723,6 +1043,206 @@ func (m *BumpClusterVersionResponse) Unmarshal(dAtA []byte) error { } return nil } +func (m *GCReplicasRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMigration + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GCReplicasRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GCReplicasRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipMigration(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMigration + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *GCReplicasResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMigration + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: GCReplicasResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: GCReplicasResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipMigration(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMigration + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *FlushAllEnginesRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMigration + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: FlushAllEnginesRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: FlushAllEnginesRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipMigration(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMigration + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *FlushAllEnginesResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowMigration + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: FlushAllEnginesResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: FlushAllEnginesResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipMigration(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthMigration + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipMigration(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 @@ -829,11 +1349,11 @@ var ( ) func init() { - proto.RegisterFile("server/serverpb/migration.proto", fileDescriptor_migration_8dfeb6fcf9144e4c) + proto.RegisterFile("server/serverpb/migration.proto", fileDescriptor_migration_a15deb200006b6c7) } -var fileDescriptor_migration_8dfeb6fcf9144e4c = []byte{ - // 278 bytes of a gzipped FileDescriptorProto +var fileDescriptor_migration_a15deb200006b6c7 = []byte{ + // 366 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2f, 0x4e, 0x2d, 0x2a, 0x4b, 0x2d, 0xd2, 0x87, 0x50, 0x05, 0x49, 0xfa, 0xb9, 0x99, 0xe9, 0x45, 0x89, 0x25, 0x99, 0xf9, 0x79, 0x7a, 0x05, 0x45, 0xf9, 0x25, 0xf9, 0x42, 0x92, 0xc9, 0xf9, 0xc9, 0xd9, 0x45, 0xf9, 0x89, @@ -844,12 +1364,17 @@ var fileDescriptor_migration_8dfeb6fcf9144e4c = []byte{ 0xd3, 0x2f, 0xc1, 0xa8, 0xc0, 0xa8, 0xc1, 0x6d, 0xa4, 0xa1, 0x87, 0x70, 0x01, 0xaa, 0x85, 0x7a, 0x68, 0x26, 0xf1, 0x25, 0xa3, 0xf0, 0x95, 0xd4, 0xb8, 0x54, 0xf0, 0xdb, 0x5c, 0x5c, 0x90, 0x9f, 0x57, 0x9c, 0xaa, 0x94, 0xc7, 0x25, 0xe9, 0x54, 0x9a, 0x5b, 0x40, 0x37, 0x77, 0xc9, 0x70, 0x49, - 0x61, 0xb3, 0x0f, 0xe2, 0x1a, 0xa3, 0xad, 0x4c, 0x5c, 0x9c, 0xbe, 0xb0, 0x48, 0x10, 0x5a, 0xc8, - 0xc8, 0x25, 0x83, 0xcf, 0x13, 0x42, 0x76, 0x7a, 0x38, 0x23, 0x48, 0x8f, 0x88, 0x70, 0x97, 0xb2, - 0x27, 0x5b, 0x3f, 0x34, 0xf4, 0x18, 0x84, 0x9a, 0x19, 0xb9, 0x84, 0x30, 0x3d, 0x24, 0x64, 0x82, - 0xc7, 0x64, 0x9c, 0xe1, 0x2d, 0x65, 0x4a, 0xa2, 0x2e, 0x98, 0x2b, 0x9c, 0xb4, 0x4e, 0x3c, 0x94, - 0x63, 0x38, 0xf1, 0x48, 0x8e, 0xf1, 0xc2, 0x23, 0x39, 0xc6, 0x1b, 0x8f, 0xe4, 0x18, 0x1f, 0x3c, - 0x92, 0x63, 0x9c, 0xf0, 0x58, 0x8e, 0xe1, 0xc2, 0x63, 0x39, 0x86, 0x1b, 0x8f, 0xe5, 0x18, 0xa2, - 0x38, 0x60, 0x06, 0x25, 0xb1, 0x81, 0x93, 0xa6, 0x31, 0x20, 0x00, 0x00, 0xff, 0xff, 0x08, 0xa5, - 0xf0, 0xc3, 0xfe, 0x02, 0x00, 0x00, + 0x61, 0xb3, 0x0f, 0xea, 0x1a, 0x61, 0x2e, 0x41, 0x77, 0xe7, 0xa0, 0xd4, 0x82, 0x9c, 0xcc, 0xe4, + 0xc4, 0x62, 0xa8, 0x2b, 0x94, 0x44, 0xb8, 0x84, 0x90, 0x05, 0xa1, 0x4a, 0x25, 0xb8, 0xc4, 0xdc, + 0x72, 0x4a, 0x8b, 0x33, 0x1c, 0x73, 0x72, 0x5c, 0xf3, 0xd2, 0x33, 0xf3, 0x52, 0xe1, 0xea, 0x25, + 0xb9, 0xc4, 0x31, 0x64, 0x20, 0x9a, 0x8c, 0xe6, 0xb1, 0x70, 0x71, 0xfa, 0xc2, 0x22, 0x59, 0x68, + 0x21, 0x23, 0x97, 0x0c, 0xbe, 0x40, 0x12, 0xb2, 0xd3, 0xc3, 0x99, 0x00, 0xf4, 0x88, 0x88, 0x57, + 0x29, 0x7b, 0xb2, 0xf5, 0x43, 0x3d, 0xc9, 0x20, 0xd4, 0xcc, 0xc8, 0x25, 0x84, 0x19, 0x60, 0x42, + 0x26, 0x78, 0x4c, 0xc6, 0x19, 0x9f, 0x52, 0xa6, 0x24, 0xea, 0x82, 0xbb, 0xa2, 0x8a, 0x8b, 0x1f, + 0x2d, 0x48, 0x85, 0x0c, 0xf1, 0x98, 0x85, 0x3d, 0x62, 0xa4, 0x8c, 0x48, 0xd1, 0x02, 0xb7, 0x3b, + 0x9b, 0x8b, 0x0b, 0x11, 0xfd, 0x42, 0x3a, 0x78, 0xcc, 0xc0, 0x48, 0x3a, 0x52, 0xba, 0x44, 0xaa, + 0x86, 0x59, 0xe6, 0xa4, 0x75, 0xe2, 0xa1, 0x1c, 0xc3, 0x89, 0x47, 0x72, 0x8c, 0x17, 0x1e, 0xc9, + 0x31, 0xde, 0x78, 0x24, 0xc7, 0xf8, 0xe0, 0x91, 0x1c, 0xe3, 0x84, 0xc7, 0x72, 0x0c, 0x17, 0x1e, + 0xcb, 0x31, 0xdc, 0x78, 0x2c, 0xc7, 0x10, 0xc5, 0x01, 0x33, 0x20, 0x89, 0x0d, 0x9c, 0xc7, 0x8d, + 0x01, 0x01, 0x00, 0x00, 0xff, 0xff, 0xc5, 0x1c, 0x4a, 0x9f, 0x47, 0x04, 0x00, 0x00, } diff --git a/pkg/server/serverpb/migration.proto b/pkg/server/serverpb/migration.proto index 871b00dc8543..a5a63d5a91ce 100644 --- a/pkg/server/serverpb/migration.proto +++ b/pkg/server/serverpb/migration.proto @@ -25,8 +25,8 @@ message ValidateTargetClusterVersionRequest { message ValidateTargetClusterVersionResponse { } -// BumpClusterVersionRequest is used to inform a given node of a cluster version -// bump. +// BumpClusterVersionRequest is used to inform the tharget node of a cluster +// version bump. message BumpClusterVersionRequest { clusterversion.ClusterVersion cluster_version = 1; } @@ -34,6 +34,20 @@ message BumpClusterVersionRequest { // BumpClusterVersionResponse is the response to an BumpClusterVersionRequest. message BumpClusterVersionResponse { } +// GCReplicasRequest is used to instruct the target node to process all GC-able +// replicas. +message GCReplicasRequest{} + +// GCReplicasResponse is the response to a GCReplicasRequest. +message GCReplicasResponse{} + +// FlushAllEnginesRequest is used to instruct the target node to flush all its +// engines. +message FlushAllEnginesRequest{} + +// FlushAllEnginesResponse is the response to a FlushAllEnginesRequest. +message FlushAllEnginesResponse{} + service Migration { // ValidateTargetClusterVersion is used to verify that the target node is // running a binary that's able to support the specified cluster version. @@ -42,7 +56,7 @@ service Migration { // node's minimum supported version <= version <= node's binary version rpc ValidateTargetClusterVersion(ValidateTargetClusterVersionRequest) returns (ValidateTargetClusterVersionResponse) { } - // BumpClusterVersion is used to inform a given node of a cluster version + // BumpClusterVersion is used to inform the target node of a cluster version // bump. The node is responsible for durably persisting the message and // enabling the corresponding version gates. // @@ -50,4 +64,12 @@ service Migration { // which checks to see that all nodes in the cluster are running binaries // that would be able to support the intended version bump. rpc BumpClusterVersion(BumpClusterVersionRequest) returns (BumpClusterVersionResponse) { } + + // FlushAllEngines is used to instruct the target node to flush all its + // engines. + rpc FlushAllEngines (FlushAllEnginesRequest) returns (FlushAllEnginesResponse) { } + + // GCReplicas is used to instruct the target node to process all GC-able + // replicas. + rpc GCReplicas (GCReplicasRequest) returns (GCReplicasResponse) { } } diff --git a/pkg/sql/catalog/bootstrap/metadata.go b/pkg/sql/catalog/bootstrap/metadata.go index 3bd9518253e5..981b21cbb028 100644 --- a/pkg/sql/catalog/bootstrap/metadata.go +++ b/pkg/sql/catalog/bootstrap/metadata.go @@ -342,6 +342,10 @@ func addSystemDescriptorsToSchema(target *MetadataSchema) { target.AddDescriptor(keys.SystemDatabaseID, systemschema.ScheduledJobsTable) target.AddDescriptor(keys.SystemDatabaseID, systemschema.SqllivenessTable) + + // Tables introduced in 21.1 + + target.AddDescriptor(keys.SystemDatabaseID, systemschema.MigrationsTable) } // addSplitIDs adds a split point for each of the PseudoTableIDs to the supplied diff --git a/pkg/sql/catalog/descpb/privilege.go b/pkg/sql/catalog/descpb/privilege.go index f59406574fca..c128985acbf2 100644 --- a/pkg/sql/catalog/descpb/privilege.go +++ b/pkg/sql/catalog/descpb/privilege.go @@ -435,6 +435,7 @@ var SystemAllowedPrivileges = map[ID]privilege.List{ keys.StatementDiagnosticsTableID: privilege.ReadWriteData, keys.ScheduledJobsTableID: privilege.ReadWriteData, keys.SqllivenessID: privilege.ReadWriteData, + keys.MigrationsID: privilege.ReadWriteData, } // SetOwner sets the owner of the privilege descriptor to the provided string. diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index 501e904b05a6..9c17f144bdad 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -342,6 +342,18 @@ CREATE TABLE system.sqlliveness ( expiration DECIMAL NOT NULL, FAMILY fam0_session_id_expiration (session_id, expiration) )` + + MigrationsTableSchema = ` +CREATE TABLE system.migrations ( + version STRING NOT NULL, + status STRING NOT NULL, + description STRING NOT NULL, + start TIMESTAMP NOT NULL DEFAULT now():::TIMESTAMP, + completed TIMESTAMP NULL, + progress STRING NULL, + CONSTRAINT "primary" PRIMARY KEY (version ASC), + FAMILY "primary" (version, status, description, start, completed, progress) +)` ) func pk(name string) descpb.IndexDescriptor { @@ -1684,6 +1696,41 @@ var ( FormatVersion: descpb.InterleavedFormatVersion, NextMutationID: 1, }) + + // MigrationsTable is the descriptor for the migrations table. It contains + // details for all past and ongoing migrations. + MigrationsTable = tabledesc.NewImmutable(descpb.TableDescriptor{ + Name: "migrations", + ID: keys.MigrationsID, + ParentID: keys.SystemDatabaseID, + UnexposedParentSchemaID: keys.PublicSchemaID, + Version: 1, + Columns: []descpb.ColumnDescriptor{ + {Name: "version", ID: 1, Type: types.String}, + {Name: "status", ID: 2, Type: types.String}, + {Name: "description", ID: 3, Type: types.String}, + {Name: "start", ID: 4, Type: types.Timestamp, DefaultExpr: &nowString}, + {Name: "completed", ID: 5, Type: types.Timestamp, Nullable: true}, + {Name: "progress", ID: 6, Type: types.String, Nullable: true}, + }, + NextColumnID: 7, + Families: []descpb.ColumnFamilyDescriptor{ + { + Name: "primary", + ID: 0, + ColumnNames: []string{ + "version", "status", "description", "start", "completed", "progress", + }, + ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 5, 6}}, + }, + NextFamilyID: 1, + PrimaryIndex: pk("version"), + NextIndexID: 2, + Privileges: descpb.NewCustomSuperuserPrivilegeDescriptor( + descpb.SystemAllowedPrivileges[keys.MigrationsID], security.NodeUserName()), + FormatVersion: descpb.InterleavedFormatVersion, + NextMutationID: 1, + }) ) // newCommentPrivilegeDescriptor returns a privilege descriptor for comment table diff --git a/pkg/sql/logictest/testdata/logic_test/distsql_agg b/pkg/sql/logictest/testdata/logic_test/distsql_agg index 12f1eefede7f..ca71294f9f19 100644 --- a/pkg/sql/logictest/testdata/logic_test/distsql_agg +++ b/pkg/sql/logictest/testdata/logic_test/distsql_agg @@ -575,7 +575,7 @@ INSERT INTO statistics_agg_test SELECT y, y%10 FROM generate_series(1, 100) AS y query FFF SELECT corr(y, x)::decimal, covar_pop(y, x)::decimal, covar_samp(y, x)::decimal FROM statistics_agg_test ---- -0.045228963191363145 3.75 3.787878787878788 +0.04522896319136315 3.75 3.787878787878788 query FFF SELECT regr_intercept(y, x), regr_r2(y, x), regr_slope(y, x) FROM statistics_agg_test diff --git a/pkg/sql/logictest/testdata/logic_test/float b/pkg/sql/logictest/testdata/logic_test/float index 7ec53f1b5c35..f26f6857aae9 100644 --- a/pkg/sql/logictest/testdata/logic_test/float +++ b/pkg/sql/logictest/testdata/logic_test/float @@ -138,4 +138,4 @@ RESET extra_float_digits query FFF SELECT -0.1234567890123456, 123456789012345.6, 1234567.890123456 ---- --0.1234567890123457 123456789012345.7 1234567.890123457 +-0.1234567890123456 123456789012345.6 1234567.890123456 diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 67c3e2b02046..1f9a5e03c132 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -502,13 +502,13 @@ defaultdb pg_extension geometry_columns public SELECT defaultdb pg_extension spatial_ref_sys public SELECT defaultdb public NULL root ALL defaultdb public NULL admin ALL -postgres pg_extension NULL admin ALL postgres pg_extension NULL root ALL +postgres pg_extension NULL admin ALL postgres pg_extension geography_columns public SELECT postgres pg_extension geometry_columns public SELECT postgres pg_extension spatial_ref_sys public SELECT -postgres public NULL admin ALL postgres public NULL root ALL +postgres public NULL admin ALL system pg_extension NULL admin GRANT system pg_extension NULL root GRANT system pg_extension geography_columns public SELECT @@ -516,261 +516,271 @@ system pg_extension geometry_columns public SELECT system pg_extension spatial_ref_sys public SELECT system public NULL root GRANT system public NULL admin GRANT -system public comments admin DELETE -system public comments root SELECT -system public comments root UPDATE -system public comments root INSERT -system public comments root GRANT system public comments root DELETE system public comments public SELECT -system public comments admin UPDATE system public comments admin SELECT -system public comments admin GRANT system public comments admin INSERT +system public comments admin GRANT +system public comments root GRANT +system public comments root INSERT +system public comments admin DELETE +system public comments root UPDATE +system public comments admin UPDATE +system public comments root SELECT system public descriptor admin GRANT system public descriptor admin SELECT system public descriptor root GRANT system public descriptor root SELECT system public eventlog admin GRANT -system public eventlog admin UPDATE -system public eventlog admin SELECT +system public eventlog root SELECT +system public eventlog root GRANT system public eventlog root UPDATE -system public eventlog root INSERT system public eventlog root DELETE -system public eventlog root SELECT +system public eventlog root INSERT +system public eventlog admin SELECT system public eventlog admin INSERT -system public eventlog root GRANT system public eventlog admin DELETE -system public jobs root DELETE +system public eventlog admin UPDATE system public jobs admin DELETE -system public jobs root GRANT -system public jobs admin GRANT +system public jobs admin INSERT +system public jobs root UPDATE +system public jobs root SELECT system public jobs root INSERT system public jobs admin SELECT +system public jobs root GRANT +system public jobs root DELETE system public jobs admin UPDATE -system public jobs root UPDATE -system public jobs root SELECT -system public jobs admin INSERT -system public lease admin SELECT +system public jobs admin GRANT +system public lease admin DELETE system public lease root UPDATE system public lease root INSERT system public lease root GRANT -system public lease root SELECT +system public lease root DELETE system public lease admin UPDATE +system public lease admin SELECT system public lease admin INSERT +system public lease root SELECT system public lease admin GRANT -system public lease admin DELETE -system public lease root DELETE -system public locations root INSERT -system public locations admin DELETE system public locations admin GRANT +system public locations root SELECT +system public locations root GRANT +system public locations root DELETE +system public locations admin DELETE system public locations admin INSERT system public locations admin SELECT -system public locations admin UPDATE -system public locations root DELETE -system public locations root GRANT -system public locations root SELECT system public locations root UPDATE -system public namespace admin GRANT -system public namespace root GRANT +system public locations root INSERT +system public locations admin UPDATE +system public migrations admin GRANT +system public migrations admin DELETE +system public migrations admin INSERT +system public migrations admin UPDATE +system public migrations root DELETE +system public migrations root GRANT +system public migrations root INSERT +system public migrations root UPDATE +system public migrations admin SELECT +system public migrations root SELECT system public namespace root SELECT +system public namespace admin GRANT system public namespace admin SELECT -system public namespace2 admin SELECT +system public namespace root GRANT system public namespace2 root SELECT -system public namespace2 root GRANT system public namespace2 admin GRANT -system public protected_ts_meta root SELECT +system public namespace2 admin SELECT +system public namespace2 root GRANT +system public protected_ts_meta root GRANT system public protected_ts_meta admin GRANT +system public protected_ts_meta root SELECT system public protected_ts_meta admin SELECT -system public protected_ts_meta root GRANT -system public protected_ts_records root SELECT +system public protected_ts_records root GRANT system public protected_ts_records admin SELECT system public protected_ts_records admin GRANT -system public protected_ts_records root GRANT -system public rangelog root SELECT +system public protected_ts_records root SELECT +system public rangelog root UPDATE +system public rangelog root INSERT system public rangelog admin INSERT -system public rangelog admin UPDATE +system public rangelog root SELECT system public rangelog admin GRANT -system public rangelog root DELETE -system public rangelog root UPDATE system public rangelog admin DELETE -system public rangelog admin SELECT system public rangelog root GRANT -system public rangelog root INSERT +system public rangelog admin SELECT +system public rangelog admin UPDATE +system public rangelog root DELETE system public replication_constraint_stats root UPDATE -system public replication_constraint_stats admin INSERT -system public replication_constraint_stats admin SELECT system public replication_constraint_stats admin GRANT -system public replication_constraint_stats root DELETE -system public replication_constraint_stats root GRANT -system public replication_constraint_stats root INSERT system public replication_constraint_stats root SELECT -system public replication_constraint_stats admin DELETE +system public replication_constraint_stats root DELETE system public replication_constraint_stats admin UPDATE -system public replication_critical_localities admin UPDATE -system public replication_critical_localities admin INSERT -system public replication_critical_localities admin SELECT +system public replication_constraint_stats admin SELECT +system public replication_constraint_stats admin INSERT +system public replication_constraint_stats admin DELETE +system public replication_constraint_stats root INSERT +system public replication_constraint_stats root GRANT system public replication_critical_localities admin DELETE +system public replication_critical_localities admin UPDATE system public replication_critical_localities root DELETE -system public replication_critical_localities root UPDATE -system public replication_critical_localities root SELECT +system public replication_critical_localities root GRANT +system public replication_critical_localities admin SELECT system public replication_critical_localities root INSERT +system public replication_critical_localities admin INSERT system public replication_critical_localities admin GRANT -system public replication_critical_localities root GRANT -system public replication_stats root DELETE +system public replication_critical_localities root SELECT +system public replication_critical_localities root UPDATE +system public replication_stats root SELECT system public replication_stats admin INSERT -system public replication_stats root UPDATE -system public replication_stats admin UPDATE system public replication_stats admin GRANT system public replication_stats admin SELECT -system public replication_stats root INSERT system public replication_stats admin DELETE -system public replication_stats root SELECT +system public replication_stats root DELETE system public replication_stats root GRANT -system public reports_meta root INSERT -system public reports_meta root SELECT -system public reports_meta root UPDATE +system public replication_stats root UPDATE +system public replication_stats admin UPDATE +system public replication_stats root INSERT system public reports_meta admin UPDATE -system public reports_meta admin SELECT -system public reports_meta root GRANT -system public reports_meta admin INSERT -system public reports_meta admin DELETE system public reports_meta admin GRANT +system public reports_meta admin INSERT +system public reports_meta root SELECT +system public reports_meta root INSERT +system public reports_meta root GRANT system public reports_meta root DELETE +system public reports_meta admin SELECT +system public reports_meta root UPDATE +system public reports_meta admin DELETE +system public role_members root UPDATE +system public role_members root SELECT system public role_members root INSERT system public role_members root GRANT system public role_members root DELETE system public role_members admin UPDATE system public role_members admin SELECT +system public role_members admin INSERT system public role_members admin GRANT system public role_members admin DELETE -system public role_members admin INSERT -system public role_members root SELECT -system public role_members root UPDATE -system public role_options root SELECT system public role_options root UPDATE system public role_options admin INSERT -system public role_options admin SELECT system public role_options admin UPDATE -system public role_options admin GRANT -system public role_options admin DELETE system public role_options root DELETE system public role_options root GRANT system public role_options root INSERT -system public scheduled_jobs admin GRANT -system public scheduled_jobs root UPDATE -system public scheduled_jobs admin DELETE +system public role_options admin SELECT +system public role_options admin GRANT +system public role_options admin DELETE +system public role_options root SELECT system public scheduled_jobs root INSERT +system public scheduled_jobs admin GRANT +system public scheduled_jobs admin SELECT +system public scheduled_jobs admin UPDATE system public scheduled_jobs root GRANT system public scheduled_jobs root DELETE -system public scheduled_jobs admin UPDATE -system public scheduled_jobs admin SELECT system public scheduled_jobs admin INSERT +system public scheduled_jobs root UPDATE system public scheduled_jobs root SELECT +system public scheduled_jobs admin DELETE system public settings admin SELECT +system public settings root INSERT system public settings root SELECT system public settings root UPDATE -system public settings admin INSERT system public settings root DELETE system public settings admin DELETE +system public settings root GRANT system public settings admin GRANT +system public settings admin INSERT system public settings admin UPDATE -system public settings root GRANT -system public settings root INSERT -system public sqlliveness root GRANT -system public sqlliveness admin DELETE system public sqlliveness admin INSERT -system public sqlliveness admin SELECT -system public sqlliveness root INSERT system public sqlliveness root SELECT -system public sqlliveness root UPDATE -system public sqlliveness admin GRANT -system public sqlliveness admin UPDATE +system public sqlliveness root GRANT system public sqlliveness root DELETE +system public sqlliveness root INSERT +system public sqlliveness admin UPDATE +system public sqlliveness admin SELECT +system public sqlliveness admin GRANT +system public sqlliveness root UPDATE +system public sqlliveness admin DELETE system public statement_bundle_chunks admin DELETE -system public statement_bundle_chunks admin GRANT -system public statement_bundle_chunks admin INSERT -system public statement_bundle_chunks admin UPDATE -system public statement_bundle_chunks root DELETE -system public statement_bundle_chunks root INSERT system public statement_bundle_chunks root SELECT system public statement_bundle_chunks root UPDATE -system public statement_bundle_chunks admin SELECT system public statement_bundle_chunks root GRANT -system public statement_diagnostics admin GRANT +system public statement_bundle_chunks root DELETE +system public statement_bundle_chunks admin UPDATE +system public statement_bundle_chunks admin SELECT +system public statement_bundle_chunks admin INSERT +system public statement_bundle_chunks root INSERT +system public statement_bundle_chunks admin GRANT system public statement_diagnostics root UPDATE -system public statement_diagnostics root SELECT system public statement_diagnostics root INSERT system public statement_diagnostics root GRANT -system public statement_diagnostics admin DELETE system public statement_diagnostics root DELETE system public statement_diagnostics admin UPDATE -system public statement_diagnostics admin INSERT system public statement_diagnostics admin SELECT -system public statement_diagnostics_requests admin GRANT -system public statement_diagnostics_requests root UPDATE -system public statement_diagnostics_requests root SELECT -system public statement_diagnostics_requests root GRANT -system public statement_diagnostics_requests root DELETE -system public statement_diagnostics_requests admin UPDATE +system public statement_diagnostics admin DELETE +system public statement_diagnostics admin INSERT +system public statement_diagnostics admin GRANT +system public statement_diagnostics root SELECT +system public statement_diagnostics_requests admin DELETE system public statement_diagnostics_requests admin INSERT system public statement_diagnostics_requests admin SELECT -system public statement_diagnostics_requests admin DELETE +system public statement_diagnostics_requests admin UPDATE +system public statement_diagnostics_requests root DELETE +system public statement_diagnostics_requests root GRANT system public statement_diagnostics_requests root INSERT -system public table_statistics admin UPDATE +system public statement_diagnostics_requests root SELECT +system public statement_diagnostics_requests root UPDATE +system public statement_diagnostics_requests admin GRANT +system public table_statistics root SELECT system public table_statistics admin SELECT system public table_statistics admin GRANT -system public table_statistics root SELECT -system public table_statistics root INSERT -system public table_statistics root GRANT system public table_statistics admin DELETE -system public table_statistics root DELETE system public table_statistics admin INSERT system public table_statistics root UPDATE +system public table_statistics admin UPDATE +system public table_statistics root DELETE +system public table_statistics root INSERT +system public table_statistics root GRANT +system public tenants admin GRANT system public tenants root SELECT system public tenants root GRANT -system public tenants admin GRANT system public tenants admin SELECT -system public ui admin SELECT -system public ui root GRANT -system public ui admin INSERT -system public ui root DELETE system public ui admin GRANT -system public ui root SELECT system public ui root UPDATE +system public ui root SELECT +system public ui admin INSERT system public ui root INSERT +system public ui root GRANT +system public ui admin SELECT system public ui admin UPDATE system public ui admin DELETE -system public users admin DELETE +system public ui root DELETE +system public users root UPDATE +system public users root SELECT system public users root INSERT -system public users root GRANT system public users root DELETE -system public users root UPDATE system public users admin UPDATE +system public users admin SELECT system public users admin INSERT +system public users admin DELETE +system public users root GRANT system public users admin GRANT -system public users root SELECT -system public users admin SELECT -system public web_sessions admin DELETE -system public web_sessions admin SELECT -system public web_sessions admin GRANT -system public web_sessions admin INSERT system public web_sessions root SELECT +system public web_sessions admin DELETE system public web_sessions root INSERT -system public web_sessions root GRANT +system public web_sessions admin INSERT +system public web_sessions admin SELECT +system public web_sessions admin UPDATE system public web_sessions root DELETE +system public web_sessions root GRANT system public web_sessions root UPDATE -system public web_sessions admin UPDATE -system public zones admin SELECT -system public zones admin DELETE +system public web_sessions admin GRANT +system public zones admin GRANT system public zones admin INSERT -system public zones admin UPDATE +system public zones admin SELECT system public zones root DELETE -system public zones root UPDATE -system public zones root INSERT system public zones root GRANT +system public zones root INSERT system public zones root SELECT -system public zones admin GRANT +system public zones root UPDATE +system public zones admin UPDATE +system public zones admin DELETE test pg_extension NULL admin ALL test pg_extension NULL root ALL test pg_extension geography_columns public SELECT @@ -1130,6 +1140,11 @@ system public locations root GRA system public locations root INSERT system public locations root SELECT system public locations root UPDATE +system public migrations root DELETE +system public migrations root GRANT +system public migrations root INSERT +system public migrations root SELECT +system public migrations root UPDATE system public namespace root GRANT system public namespace root SELECT system public namespace2 root GRANT diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index eff1fe26f269..21ffec7799c9 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -704,6 +704,7 @@ system public statement_diagnostics_requests BASE TABLE system public statement_diagnostics BASE TABLE YES 1 system public scheduled_jobs BASE TABLE YES 1 system public sqlliveness BASE TABLE YES 1 +system public migrations BASE TABLE YES 1 statement ok ALTER TABLE other_db.xyz ADD COLUMN j INT @@ -789,6 +790,11 @@ system public 630200280_21_2_not_null system system public 630200280_21_3_not_null system public locations CHECK NO NO system public 630200280_21_4_not_null system public locations CHECK NO NO system public primary system public locations PRIMARY KEY NO NO +system public 630200280_40_1_not_null system public migrations CHECK NO NO +system public 630200280_40_2_not_null system public migrations CHECK NO NO +system public 630200280_40_3_not_null system public migrations CHECK NO NO +system public 630200280_40_4_not_null system public migrations CHECK NO NO +system public primary system public migrations PRIMARY KEY NO NO system public 630200280_2_1_not_null system public namespace CHECK NO NO system public 630200280_2_2_not_null system public namespace CHECK NO NO system public primary system public namespace PRIMARY KEY NO NO @@ -1019,6 +1025,10 @@ system public 630200280_37_9_not_null executor_type I system public 630200280_39_1_not_null session_id IS NOT NULL system public 630200280_39_2_not_null expiration IS NOT NULL system public 630200280_3_1_not_null id IS NOT NULL +system public 630200280_40_1_not_null version IS NOT NULL +system public 630200280_40_2_not_null status IS NOT NULL +system public 630200280_40_3_not_null description IS NOT NULL +system public 630200280_40_4_not_null start IS NOT NULL system public 630200280_4_1_not_null username IS NOT NULL system public 630200280_4_3_not_null isRole IS NOT NULL system public 630200280_5_1_not_null id IS NOT NULL @@ -1048,6 +1058,7 @@ system public lease nodeID sy system public lease version system public primary system public locations localityKey system public primary system public locations localityValue system public primary +system public migrations version system public primary system public namespace name system public primary system public namespace parentID system public primary system public namespace2 name system public primary @@ -1212,6 +1223,12 @@ system public locations latitude system public locations localityKey 1 system public locations localityValue 2 system public locations longitude 4 +system public migrations completed 5 +system public migrations description 3 +system public migrations progress 6 +system public migrations start 4 +system public migrations status 2 +system public migrations version 1 system public namespace id 3 system public namespace name 2 system public namespace parentID 1 @@ -1853,6 +1870,16 @@ NULL root system public locations NULL root system public locations INSERT NULL NO NULL root system public locations SELECT NULL YES NULL root system public locations UPDATE NULL NO +NULL admin system public migrations DELETE NULL NO +NULL admin system public migrations GRANT NULL NO +NULL admin system public migrations INSERT NULL NO +NULL admin system public migrations SELECT NULL YES +NULL admin system public migrations UPDATE NULL NO +NULL root system public migrations DELETE NULL NO +NULL root system public migrations GRANT NULL NO +NULL root system public migrations INSERT NULL NO +NULL root system public migrations SELECT NULL YES +NULL root system public migrations UPDATE NULL NO NULL admin system public namespace GRANT NULL NO NULL admin system public namespace SELECT NULL YES NULL root system public namespace GRANT NULL NO @@ -2434,6 +2461,16 @@ NULL root system public sqlliveness NULL root system public sqlliveness INSERT NULL NO NULL root system public sqlliveness SELECT NULL YES NULL root system public sqlliveness UPDATE NULL NO +NULL admin system public migrations DELETE NULL NO +NULL admin system public migrations GRANT NULL NO +NULL admin system public migrations INSERT NULL NO +NULL admin system public migrations SELECT NULL YES +NULL admin system public migrations UPDATE NULL NO +NULL root system public migrations DELETE NULL NO +NULL root system public migrations GRANT NULL NO +NULL root system public migrations INSERT NULL NO +NULL root system public migrations SELECT NULL YES +NULL root system public migrations UPDATE NULL NO statement ok CREATE TABLE other_db.xyz (i INT) diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 13b27350bf65..29bfd4ede2ec 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -721,6 +721,7 @@ indexrelid indrelid indnatts indisunique indisprimary indisexclusion indim 2008917578 37 1 false false false false false true false false true false 5 0 0 2 NULL NULL 2101708905 5 1 true true false true false true false false true false 1 0 0 2 NULL NULL 2148104569 21 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL +2268653844 40 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL 2361445172 8 1 true true false true false true false false true false 1 0 0 2 NULL NULL 2407840836 24 3 true true false true false true false false true false 1 2 3 0 0 0 0 0 0 2 2 2 NULL NULL 2621181440 15 2 false false false false false true false false true false 2 3 3403232968 0 0 0 2 2 NULL NULL @@ -777,6 +778,7 @@ indexrelid operator_argument_type_oid operator_argument_position 2101708905 0 1 2148104569 0 1 2148104569 0 2 +2268653844 0 1 2361445172 0 1 2407840836 0 1 2407840836 0 2 diff --git a/pkg/sql/logictest/testdata/logic_test/ranges b/pkg/sql/logictest/testdata/logic_test/ranges index 018de6407d4a..0d88ba9f18f4 100644 --- a/pkg/sql/logictest/testdata/logic_test/ranges +++ b/pkg/sql/logictest/testdata/logic_test/ranges @@ -309,7 +309,8 @@ start_key start_pretty end_key [172] /Table/36 [173] /Table/37 system statement_diagnostics · {1} 1 [173] /Table/37 [174] /Table/38 system scheduled_jobs · {1} 1 [174] /Table/38 [175] /Table/39 · · · {1} 1 -[175] /Table/39 [189 137] /Table/53/1 system sqlliveness · {1} 1 +[175] /Table/39 [176] /Table/40 system sqlliveness · {1} 1 +[176] /Table/40 [189 137] /Table/53/1 system migrations · {1} 1 [189 137] /Table/53/1 [189 137 137] /Table/53/1/1 test t · {1} 1 [189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3 [189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1 @@ -367,7 +368,8 @@ start_key start_pretty end_key [172] /Table/36 [173] /Table/37 system statement_diagnostics · {1} 1 [173] /Table/37 [174] /Table/38 system scheduled_jobs · {1} 1 [174] /Table/38 [175] /Table/39 · · · {1} 1 -[175] /Table/39 [189 137] /Table/53/1 system sqlliveness · {1} 1 +[175] /Table/39 [176] /Table/40 system sqlliveness · {1} 1 +[176] /Table/40 [189 137] /Table/53/1 system migrations · {1} 1 [189 137] /Table/53/1 [189 137 137] /Table/53/1/1 test t · {1} 1 [189 137 137] /Table/53/1/1 [189 137 141 137] /Table/53/1/5/1 test t · {3,4} 3 [189 137 141 137] /Table/53/1/5/1 [189 137 141 138] /Table/53/1/5/2 test t · {1,2,3} 1 diff --git a/pkg/sql/logictest/testdata/logic_test/schema b/pkg/sql/logictest/testdata/logic_test/schema index 88a5c32fdd1c..27bb3b77b2c3 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema +++ b/pkg/sql/logictest/testdata/logic_test/schema @@ -774,5 +774,3 @@ ALTER TABLE public_table_to_rename RENAME TO public.renamed_public_table; statement error pq: relation "public_table_exists" already exists ALTER TABLE renamed_public_table RENAME TO public_table_exists; - - diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index 834658da7a69..fe1d0c211499 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -204,6 +204,7 @@ public jobs table NULL NULL public web_sessions table NULL NULL public table_statistics table NULL NULL public locations table NULL NULL +public migrations table NULL NULL public role_members table NULL NULL public comments table NULL NULL public replication_constraint_stats table NULL NULL @@ -238,6 +239,7 @@ public jobs table NULL NULL public web_sessions table NULL NULL · public table_statistics table NULL NULL · public locations table NULL NULL · +public migrations table NULL NULL · public role_members table NULL NULL · public comments table NULL NULL · public replication_constraint_stats table NULL NULL · diff --git a/pkg/sql/logictest/testdata/logic_test/system b/pkg/sql/logictest/testdata/logic_test/system index 69368b0de722..a3639132c9b3 100644 --- a/pkg/sql/logictest/testdata/logic_test/system +++ b/pkg/sql/logictest/testdata/logic_test/system @@ -16,6 +16,7 @@ public eventlog table NULL NULL public jobs table NULL NULL public lease table NULL NULL public locations table NULL NULL +public migrations table NULL NULL public namespace table NULL NULL public namespace2 table NULL NULL public protected_ts_meta table NULL NULL @@ -73,6 +74,7 @@ SELECT id FROM system.descriptor 36 37 39 +40 50 51 52 @@ -241,6 +243,16 @@ system public locations root GRANT system public locations root INSERT system public locations root SELECT system public locations root UPDATE +system public migrations admin DELETE +system public migrations admin GRANT +system public migrations admin INSERT +system public migrations admin SELECT +system public migrations admin UPDATE +system public migrations root DELETE +system public migrations root GRANT +system public migrations root INSERT +system public migrations root SELECT +system public migrations root UPDATE system public namespace admin GRANT system public namespace admin SELECT system public namespace root GRANT diff --git a/pkg/sql/logictest/testdata/logic_test/system_namespace b/pkg/sql/logictest/testdata/logic_test/system_namespace index ce32ec8c320b..6291b9c053f3 100644 --- a/pkg/sql/logictest/testdata/logic_test/system_namespace +++ b/pkg/sql/logictest/testdata/logic_test/system_namespace @@ -18,6 +18,7 @@ SELECT * FROM system.namespace 1 29 jobs 15 1 29 lease 11 1 29 locations 21 +1 29 migrations 40 1 29 namespace 2 1 29 namespace2 30 1 29 protected_ts_meta 31 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/autocommit b/pkg/sql/opt/exec/execbuilder/testdata/autocommit index b0e2fcf405f2..26a33095d51a 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/autocommit +++ b/pkg/sql/opt/exec/execbuilder/testdata/autocommit @@ -41,7 +41,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 # Multi-row insert should auto-commit. query B @@ -62,7 +62,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -86,7 +86,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 statement ok ROLLBACK @@ -110,7 +110,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 CPut, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -132,8 +132,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 2 CPut to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Insert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -156,8 +156,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 2 CPut to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -192,7 +192,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 Put, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Put, 1 EndTxn to (n1,s1):1 # Multi-row upsert should auto-commit. query B @@ -213,7 +213,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -237,7 +237,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 2 Put to (n1,s1):1 +dist sender send r36: sending batch 2 Put to (n1,s1):1 statement ok ROLLBACK @@ -261,7 +261,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -283,8 +283,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 2 Put to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 Put to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Upsert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -307,8 +307,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 2 Put to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 Put to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -343,8 +343,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -368,8 +368,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Put to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Put to (n1,s1):1 statement ok ROLLBACK @@ -393,8 +393,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Put, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Put, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -416,9 +416,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Put to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Put to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Update with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -441,9 +441,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Put to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Put to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Another way to test the scenario above: generate an error and ensure that the # mutation was not committed. @@ -478,7 +478,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # Multi-row delete should auto-commit. query B @@ -499,7 +499,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # No auto-commit inside a transaction. statement ok @@ -523,7 +523,7 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 DelRng to (n1,s1):1 +dist sender send r36: sending batch 1 DelRng to (n1,s1):1 statement ok ROLLBACK @@ -547,8 +547,8 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%PushTxn%' AND message NOT LIKE '%QueryTxn%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Del, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Del, 1 EndTxn to (n1,s1):1 # TODO(radu): allow non-side-effecting projections. query B @@ -570,9 +570,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Del to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Del to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Insert with RETURNING statement with side-effects should not auto-commit. # In this case division can (in principle) error out. @@ -595,9 +595,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 2 Del to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 2 Del to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 statement ok INSERT INTO ab VALUES (12, 0); @@ -644,9 +644,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 2 CPut to (n1,s1):1 -dist sender send r35: sending batch 2 Scan to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 2 Scan to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -667,10 +667,10 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 1 Put to (n1,s1):1 -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 1 Put to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -692,10 +692,10 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 1 Del to (n1,s1):1 -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 1 Del to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 # Test with a single cascade, which should use autocommit. statement ok @@ -719,9 +719,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 1 DelRng to (n1,s1):1 -dist sender send r35: sending batch 1 Scan to (n1,s1):1 -dist sender send r35: sending batch 1 Del, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 DelRng to (n1,s1):1 +dist sender send r36: sending batch 1 Scan to (n1,s1):1 +dist sender send r36: sending batch 1 Del, 1 EndTxn to (n1,s1):1 # ----------------------- # Multiple mutation tests @@ -749,9 +749,9 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 2 CPut to (n1,s1):1 -dist sender send r35: sending batch 2 CPut to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 query B SELECT count(*) > 0 FROM [ @@ -774,6 +774,6 @@ WHERE message LIKE '%r$rangeid: sending batch%' AND message NOT LIKE '%QueryTxn%' AND operation NOT LIKE '%async%' ---- -dist sender send r35: sending batch 2 CPut to (n1,s1):1 -dist sender send r35: sending batch 2 CPut to (n1,s1):1 -dist sender send r35: sending batch 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 2 CPut to (n1,s1):1 +dist sender send r36: sending batch 1 EndTxn to (n1,s1):1 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/delete b/pkg/sql/opt/exec/execbuilder/testdata/delete index 567afdb55101..c24b7a488188 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/delete +++ b/pkg/sql/opt/exec/execbuilder/testdata/delete @@ -231,9 +231,9 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%DelRange%' OR message LIKE '%DelRng%' ---- flow DelRange /Table/57/1 - /Table/57/2 -dist sender send r35: sending batch 1 DelRng to (n1,s1):1 +dist sender send r36: sending batch 1 DelRng to (n1,s1):1 flow DelRange /Table/57/1/601/0 - /Table/57/2 -dist sender send r35: sending batch 1 DelRng to (n1,s1):1 +dist sender send r36: sending batch 1 DelRng to (n1,s1):1 # Ensure that DelRange requests are autocommitted when DELETE FROM happens on a # chunk of fewer than 600 keys. @@ -249,7 +249,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] WHERE message LIKE '%DelRange%' OR message LIKE '%sending batch%' ---- flow DelRange /Table/57/1/5 - /Table/57/1/5/# -dist sender send r35: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 +dist sender send r36: sending batch 1 DelRng, 1 EndTxn to (n1,s1):1 # Test use of fast path when there are interleaved tables. diff --git a/pkg/sql/opt/exec/execbuilder/testdata/upsert b/pkg/sql/opt/exec/execbuilder/testdata/upsert index 137381ed74c1..077b0f93347e 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/upsert +++ b/pkg/sql/opt/exec/execbuilder/testdata/upsert @@ -340,7 +340,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] table reader Scan /Table/57/1/2{-/#} flow CPut /Table/57/1/2/0 -> /TUPLE/2:2:Int/3 flow InitPut /Table/57/2/3/0 -> /BYTES/0x8a -kv.DistSender: sending partial batch r35: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r36: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -354,7 +354,7 @@ SELECT operation, message FROM [SHOW KV TRACE FOR SESSION] table reader Scan /Table/57/1/1{-/#} flow CPut /Table/57/1/1/0 -> /TUPLE/2:2:Int/2 flow InitPut /Table/57/2/2/0 -> /BYTES/0x89 -kv.DistSender: sending partial batch r35: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r36: sending batch 1 CPut, 1 EndTxn to (n1,s1):1 flow fast path completed exec stmt rows affected: 1 @@ -371,7 +371,7 @@ table reader fetched: /kv/primary/2/v -> /3 flow Put /Table/57/1/2/0 -> /TUPLE/2:2:Int/2 flow Del /Table/57/2/3/0 flow CPut /Table/57/2/2/0 -> /BYTES/0x8a (expecting does not exist) -kv.DistSender: sending partial batch r35: sending batch 1 Put, 1 EndTxn to (n1,s1):1 +kv.DistSender: sending partial batch r36: sending batch 1 Put, 1 EndTxn to (n1,s1):1 exec stmt execution failed after 0 rows: duplicate key value (v)=(2) violates unique constraint "woo" diff --git a/pkg/sql/pgwire/pgwire_test.go b/pkg/sql/pgwire/pgwire_test.go index 3da78a8aa87a..aefb49dea557 100644 --- a/pkg/sql/pgwire/pgwire_test.go +++ b/pkg/sql/pgwire/pgwire_test.go @@ -556,7 +556,7 @@ func TestPGPreparedQuery(t *testing.T) { baseTest.Results("users", "primary", false, 1, "username", "ASC", false, false), }}, {"SHOW TABLES FROM system", []preparedQueryTest{ - baseTest.Results("public", "comments", "table", gosql.NullString{}, gosql.NullString{}).Others(28), + baseTest.Results("public", "comments", "table", gosql.NullString{}, gosql.NullString{}).Others(29), }}, {"SHOW SCHEMAS FROM system", []preparedQueryTest{ baseTest.Results("crdb_internal", gosql.NullString{}).Others(4), diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go index 98a97d1d6565..05c05e41305e 100644 --- a/pkg/sql/tests/system_table_test.go +++ b/pkg/sql/tests/system_table_test.go @@ -191,6 +191,7 @@ func TestSystemTableLiterals(t *testing.T) { {keys.StatementDiagnosticsTableID, systemschema.StatementDiagnosticsTableSchema, systemschema.StatementDiagnosticsTable}, {keys.ScheduledJobsTableID, systemschema.ScheduledJobsTableSchema, systemschema.ScheduledJobsTable}, {keys.SqllivenessID, systemschema.SqllivenessTableSchema, systemschema.SqllivenessTable}, + {keys.MigrationsID, systemschema.MigrationsTableSchema, systemschema.MigrationsTable}, } { privs := *test.pkg.Privileges gen, err := sql.CreateTestTableDescriptor( diff --git a/pkg/sql/tests/testdata/initial_keys b/pkg/sql/tests/testdata/initial_keys index 0a3eeea059ec..fede84a0c496 100644 --- a/pkg/sql/tests/testdata/initial_keys +++ b/pkg/sql/tests/testdata/initial_keys @@ -1,6 +1,6 @@ initial-keys tenant=system ---- -69 keys: +71 keys: /System/"desc-idgen" /Table/3/1/1/2/1 /Table/3/1/2/2/1 @@ -32,6 +32,7 @@ initial-keys tenant=system /Table/3/1/36/2/1 /Table/3/1/37/2/1 /Table/3/1/39/2/1 + /Table/3/1/40/2/1 /Table/5/1/0/2/1 /Table/5/1/1/2/1 /Table/5/1/16/2/1 @@ -47,6 +48,7 @@ initial-keys tenant=system /NamespaceTable/30/1/1/29/"jobs"/4/1 /NamespaceTable/30/1/1/29/"lease"/4/1 /NamespaceTable/30/1/1/29/"locations"/4/1 + /NamespaceTable/30/1/1/29/"migrations"/4/1 /NamespaceTable/30/1/1/29/"namespace"/4/1 /NamespaceTable/30/1/1/29/"namespace2"/4/1 /NamespaceTable/30/1/1/29/"protected_ts_meta"/4/1 @@ -70,7 +72,7 @@ initial-keys tenant=system /NamespaceTable/30/1/1/29/"users"/4/1 /NamespaceTable/30/1/1/29/"web_sessions"/4/1 /NamespaceTable/30/1/1/29/"zones"/4/1 -29 splits: +30 splits: /Table/11 /Table/12 /Table/13 @@ -100,10 +102,11 @@ initial-keys tenant=system /Table/37 /Table/38 /Table/39 + /Table/40 initial-keys tenant=5 ---- -60 keys: +62 keys: /Tenant/5/Table/3/1/1/2/1 /Tenant/5/Table/3/1/2/2/1 /Tenant/5/Table/3/1/3/2/1 @@ -133,6 +136,7 @@ initial-keys tenant=5 /Tenant/5/Table/3/1/36/2/1 /Tenant/5/Table/3/1/37/2/1 /Tenant/5/Table/3/1/39/2/1 + /Tenant/5/Table/3/1/40/2/1 /Tenant/5/Table/7/1/0/0 /Tenant/5/NamespaceTable/30/1/0/0/"system"/4/1 /Tenant/5/NamespaceTable/30/1/1/0/"public"/4/1 @@ -143,6 +147,7 @@ initial-keys tenant=5 /Tenant/5/NamespaceTable/30/1/1/29/"jobs"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"lease"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"locations"/4/1 + /Tenant/5/NamespaceTable/30/1/1/29/"migrations"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"namespace"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"namespace2"/4/1 /Tenant/5/NamespaceTable/30/1/1/29/"protected_ts_meta"/4/1 @@ -169,7 +174,7 @@ initial-keys tenant=5 initial-keys tenant=999 ---- -60 keys: +62 keys: /Tenant/999/Table/3/1/1/2/1 /Tenant/999/Table/3/1/2/2/1 /Tenant/999/Table/3/1/3/2/1 @@ -199,6 +204,7 @@ initial-keys tenant=999 /Tenant/999/Table/3/1/36/2/1 /Tenant/999/Table/3/1/37/2/1 /Tenant/999/Table/3/1/39/2/1 + /Tenant/999/Table/3/1/40/2/1 /Tenant/999/Table/7/1/0/0 /Tenant/999/NamespaceTable/30/1/0/0/"system"/4/1 /Tenant/999/NamespaceTable/30/1/1/0/"public"/4/1 @@ -209,6 +215,7 @@ initial-keys tenant=999 /Tenant/999/NamespaceTable/30/1/1/29/"jobs"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"lease"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"locations"/4/1 + /Tenant/999/NamespaceTable/30/1/1/29/"migrations"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"namespace"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"namespace2"/4/1 /Tenant/999/NamespaceTable/30/1/1/29/"protected_ts_meta"/4/1 diff --git a/pkg/sqlmigrations/migrations.go b/pkg/sqlmigrations/migrations.go index 5c6836272be7..07638ee2c4bb 100644 --- a/pkg/sqlmigrations/migrations.go +++ b/pkg/sqlmigrations/migrations.go @@ -330,6 +330,13 @@ var backwardCompatibleMigrations = []migrationDescriptor{ // Introduced in v20.2. name: "mark non-terminal schema change jobs with a pre-20.1 format version as failed", }, + { + // Introduced in v21.1. + name: "create new system.migrations table", + workFn: createMigrationTable, + includedInBootstrap: clusterversion.ByKey(clusterversion.MigrationTable), + newDescriptorIDs: staticIDs(keys.MigrationsID), + }, } func staticIDs( @@ -1263,6 +1270,14 @@ func createTenantsTable(ctx context.Context, r runner) error { return createSystemTable(ctx, r, systemschema.TenantsTable) } +func createMigrationTable(ctx context.Context, r runner) error { + err := createSystemTable(ctx, r, systemschema.MigrationsTable) + if err != nil { + return errors.Wrap(err, "failed to create system.migrations") + } + return nil +} + func alterSystemScheduledJobsFixTableSchema(ctx context.Context, r runner) error { setOwner := "UPDATE system.scheduled_jobs SET owner='root' WHERE owner IS NULL" asNode := sessiondata.InternalExecutorOverride{User: security.NodeUserName()} diff --git a/pkg/testutils/serverutils/BUILD.bazel b/pkg/testutils/serverutils/BUILD.bazel index 0d82321cd759..0458138882d2 100644 --- a/pkg/testutils/serverutils/BUILD.bazel +++ b/pkg/testutils/serverutils/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//pkg/rpc", "//pkg/security", "//pkg/settings/cluster", + "//pkg/storage", "//pkg/testutils/sqlutils", "//pkg/util/hlc", "//pkg/util/httputil", diff --git a/pkg/testutils/serverutils/test_server_shim.go b/pkg/testutils/serverutils/test_server_shim.go index ea4870d5b482..79ccbf7a73c3 100644 --- a/pkg/testutils/serverutils/test_server_shim.go +++ b/pkg/testutils/serverutils/test_server_shim.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/httputil" @@ -104,6 +105,8 @@ type TestServerInterface interface { // MigrationServer returns the internal *migrationServer as in interface{} MigrationServer() interface{} + Engines() []storage.Engine + // SQLServer returns the *sql.Server as an interface{}. SQLServer() interface{} diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 0ed16ca993dd..ab61ba27bd8e 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -305,6 +305,7 @@ var charts = []sectionDescription{ "distsender.rpc.initput.sent", "distsender.rpc.leaseinfo.sent", "distsender.rpc.merge.sent", + "distsender.rpc.migrate.sent", "distsender.rpc.pushtxn.sent", "distsender.rpc.put.sent", "distsender.rpc.queryintent.sent",